Commit graph

4231 commits

Author SHA1 Message Date
Cheng Lian 68b4020d0c [SPARK-16648][SQL] Make ignoreNullsExpr a child expression of First and Last
## What changes were proposed in this pull request?

Default `TreeNode.withNewChildren` implementation doesn't work for `Last` and when both constructor arguments are the same, e.g.:

```sql
LAST_VALUE(FALSE) -- The 2nd argument defaults to FALSE
LAST_VALUE(FALSE, FALSE)
LAST_VALUE(TRUE, TRUE)
```

This is because although `Last` is a unary expression, both of its constructor arguments, `child` and `ignoreNullsExpr`, are `Expression`s. When they have the same value, `TreeNode.withNewChildren` treats both of them as child nodes by mistake. `First` is also affected by this issue in exactly the same way.

This PR fixes this issue by making `ignoreNullsExpr` a child expression of `First` and `Last`.

## How was this patch tested?

New test case added in `WindowQuerySuite`.

Author: Cheng Lian <lian@databricks.com>

Closes #14295 from liancheng/spark-16648-last-value.
2016-07-25 17:22:29 +08:00
Qifan Pu 468a3c3ac5 [SPARK-16699][SQL] Fix performance bug in hash aggregate on long string keys
In the following code in `VectorizedHashMapGenerator.scala`:
```
    def hashBytes(b: String): String = {
      val hash = ctx.freshName("hash")
      s"""
         |int $result = 0;
         |for (int i = 0; i < $b.length; i++) {
         |  ${genComputeHash(ctx, s"$b[i]", ByteType, hash)}
         |  $result = ($result ^ (0x9e3779b9)) + $hash + ($result << 6) + ($result >>> 2);
         |}
       """.stripMargin
    }

```
when b=input.getBytes(), the current 2.0 code results in getBytes() being called n times, n being length of input. getBytes() involves memory copy is thus expensive and causes a performance degradation.
Fix is to evaluate getBytes() before the for loop.

Performance bug, no additional test added.

Author: Qifan Pu <qifan.pu@gmail.com>

Closes #14337 from ooq/SPARK-16699.

(cherry picked from commit d226dce12b)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2016-07-24 21:54:42 -07:00
Wenchen Fan 1221ce0402 [SPARK-16645][SQL] rename CatalogStorageFormat.serdeProperties to properties
## What changes were proposed in this pull request?

we also store data source table options in this field, it's unreasonable to call it `serdeProperties`.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14283 from cloud-fan/minor1.
2016-07-25 09:28:56 +08:00
Dongjoon Hyun cc1d2dcb61 [SPARK-16463][SQL] Support truncate option in Overwrite mode for JDBC DataFrameWriter
## What changes were proposed in this pull request?

This PR adds a boolean option, `truncate`, for `SaveMode.Overwrite` of JDBC DataFrameWriter. If this option is `true`, it try to take advantage of `TRUNCATE TABLE` instead of `DROP TABLE`. This is a trivial option, but will provide great **convenience** for BI tool users based on RDBMS tables generated by Spark.

**Goal**
- Without `CREATE/DROP` privilege, we can save dataframe to database. Sometime these are not allowed for security.
- It will preserve the existing table information, so users can add and keep some additional `INDEX` and `CONSTRAINT`s for the table.
- Sometime, `TRUNCATE` is faster than the combination of `DROP/CREATE`.

**Supported DBMS**
The following is `truncate`-option support table. Due to the different behavior of `TRUNCATE TABLE` among DBMSs, it's not always safe to use `TRUNCATE TABLE`. Spark will ignore the `truncate` option for **unknown** and **some** DBMS with **default CASCADING** behavior. Newly added JDBCDialect should implement corresponding function to support `truncate` option additionally.

Spark Dialects | `truncate` OPTION SUPPORT
---------------|-------------------------------
MySQLDialect | O
PostgresDialect | X
DB2Dialect | O
MsSqlServerDialect | O
DerbyDialect | O
OracleDialect | O

**Before (TABLE with INDEX case)**: SparkShell & MySQL CLI are interleaved intentionally.
```scala
scala> val (url, prop)=("jdbc:mysql://localhost:3306/temp?useSSL=false", new java.util.Properties)
scala> prop.setProperty("user","root")
scala> df.write.mode("overwrite").jdbc(url, "table_with_index", prop)
scala> spark.range(10).write.mode("overwrite").jdbc(url, "table_with_index", prop)
mysql> DESC table_with_index;
+-------+------------+------+-----+---------+-------+
| Field | Type       | Null | Key | Default | Extra |
+-------+------------+------+-----+---------+-------+
| id    | bigint(20) | NO   |     | NULL    |       |
+-------+------------+------+-----+---------+-------+
mysql> CREATE UNIQUE INDEX idx_id ON table_with_index(id);
mysql> DESC table_with_index;
+-------+------------+------+-----+---------+-------+
| Field | Type       | Null | Key | Default | Extra |
+-------+------------+------+-----+---------+-------+
| id    | bigint(20) | NO   | PRI | NULL    |       |
+-------+------------+------+-----+---------+-------+
scala> spark.range(10).write.mode("overwrite").jdbc(url, "table_with_index", prop)
mysql> DESC table_with_index;
+-------+------------+------+-----+---------+-------+
| Field | Type       | Null | Key | Default | Extra |
+-------+------------+------+-----+---------+-------+
| id    | bigint(20) | NO   |     | NULL    |       |
+-------+------------+------+-----+---------+-------+
```

**After (TABLE with INDEX case)**
```scala
scala> spark.range(10).write.mode("overwrite").option("truncate", true).jdbc(url, "table_with_index", prop)
mysql> DESC table_with_index;
+-------+------------+------+-----+---------+-------+
| Field | Type       | Null | Key | Default | Extra |
+-------+------------+------+-----+---------+-------+
| id    | bigint(20) | NO   | PRI | NULL    |       |
+-------+------------+------+-----+---------+-------+
```

**Error Handling**
- In case of exceptions, Spark will not retry. Users should turn off the `truncate` option.
- In case of schema change:
  - If one of the column names changes, this will raise exceptions intuitively.
  - If there exists only type difference, this will work like Append mode.

## How was this patch tested?

Pass the Jenkins tests with a updated testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14086 from dongjoon-hyun/SPARK-16410.
2016-07-24 09:25:02 +01:00
Liwei Lin d6795c7a25 [SPARK-16515][SQL][FOLLOW-UP] Fix test script on OS X/Windows...
## Problem

The current `sed` in `test_script.sh` is missing a `$`, leading to the failure of `script` test on OS X:
```
== Results ==
!== Correct Answer - 2 ==   == Spark Answer - 2 ==
![x1_y1]                    [x1]
![x2_y2]                    [x2]
```

In addition, this `script` test would also fail on systems like Windows where we couldn't be able to invoke `bash` or `echo | sed`.

## What changes were proposed in this pull request?
This patch
- fixes `sed` in `test_script.sh`
- adds command guards so that the `script` test would pass on systems like Windows

## How was this patch tested?

- Jenkins
- Manually verified tests pass on OS X

Author: Liwei Lin <lwlin7@gmail.com>

Closes #14280 from lw-lin/osx-sed.
2016-07-24 08:35:57 +01:00
Wenchen Fan 86c2752066 [SPARK-16690][TEST] rename SQLTestUtils.withTempTable to withTempView
## What changes were proposed in this pull request?

after https://github.com/apache/spark/pull/12945, we renamed the `registerTempTable` to `createTempView`, as we do create a view actually. This PR renames `SQLTestUtils.withTempTable` to reflect this change.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14318 from cloud-fan/minor4.
2016-07-23 11:39:48 -07:00
Liang-Chi Hsieh e10b8741d8 [SPARK-16622][SQL] Fix NullPointerException when the returned value of the called method in Invoke is null
## What changes were proposed in this pull request?

Currently we don't check the value returned by called method in `Invoke`. When the returned value is null and is assigned to a variable of primitive type, `NullPointerException` will be thrown.

## How was this patch tested?

Jenkins tests.

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

Closes #14259 from viirya/agg-empty-ds.
2016-07-23 10:27:16 +08:00
Jacek Laskowski e1bd70f44b [SPARK-16287][HOTFIX][BUILD][SQL] Fix annotation argument needs to be a constant
## What changes were proposed in this pull request?

Build fix for [SPARK-16287][SQL] Implement str_to_map SQL function that has introduced this compilation error:

```
/Users/jacek/dev/oss/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala:402: error: annotation argument needs to be a constant; found: "_FUNC_(text[, pairDelim, keyValueDelim]) - Creates a map after splitting the text ".+("into key/value pairs using delimiters. ").+("Default delimiters are \',\' for pairDelim and \':\' for keyValueDelim.")
    "into key/value pairs using delimiters. " +
                                              ^
```

## How was this patch tested?

Local build

Author: Jacek Laskowski <jacek@japila.pl>

Closes #14315 from jaceklaskowski/build-fix-complexTypeCreator.
2016-07-22 12:37:30 +01:00
gatorsmile 94f14b52a6 [SPARK-16556][SPARK-16559][SQL] Fix Two Bugs in Bucket Specification
### What changes were proposed in this pull request?

**Issue 1: Silent Ignorance of Bucket Specification When Creating Table Using Schema Inference**

When creating a data source table without explicit specification of schema or SELECT clause, we silently ignore the bucket specification (CLUSTERED BY... SORTED BY...) in [the code](ce3b98bae2/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala (L339-L354)).

For example,
```SQL
CREATE TABLE jsonTable
USING org.apache.spark.sql.json
OPTIONS (
  path '${tempDir.getCanonicalPath}'
)
CLUSTERED BY (inexistentColumnA) SORTED BY (inexistentColumnB) INTO 2 BUCKETS
```

This PR captures it and issues an error message.

**Issue 2: Got a run-time `java.lang.ArithmeticException` when num of buckets is set to zero.**

For example,
```SQL
CREATE TABLE t USING PARQUET
OPTIONS (PATH '${path.toString}')
CLUSTERED BY (a) SORTED BY (b) INTO 0 BUCKETS
AS SELECT 1 AS a, 2 AS b
```
The exception we got is
```
ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 1.0 (TID 2)
java.lang.ArithmeticException: / by zero
```

This PR captures the misuse and issues an appropriate error message.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14210 from gatorsmile/createTableWithoutSchema.
2016-07-22 13:27:17 +08:00
Sandeep Singh df2c6d59d0 [SPARK-16287][SQL] Implement str_to_map SQL function
## What changes were proposed in this pull request?
This PR adds `str_to_map` SQL function in order to remove Hive fallback.

## How was this patch tested?
Pass the Jenkins tests with newly added.

Author: Sandeep Singh <sandeep@techaddict.me>

Closes #13990 from techaddict/SPARK-16287.
2016-07-22 10:05:21 +08:00
Sameer Agarwal 46f80a3073 [SPARK-16334] Maintain single dictionary per row-batch in vectorized parquet reader
## What changes were proposed in this pull request?

As part of the bugfix in https://github.com/apache/spark/pull/12279, if a row batch consist of both dictionary encoded and non-dictionary encoded pages, we explicitly decode the dictionary for the values that are already dictionary encoded. Currently we reset the dictionary while reading every page that can potentially cause ` java.lang.ArrayIndexOutOfBoundsException` while decoding older pages. This patch fixes the problem by maintaining a single dictionary per row-batch in vectorized parquet reader.

## How was this patch tested?

Manual Tests against a number of hand-generated parquet files.

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

Closes #14225 from sameeragarwal/vectorized.
2016-07-21 15:34:32 -07:00
Yin Huai 9abd99b3c3 [SPARK-16656][SQL] Try to make CreateTableAsSelectSuite more stable
## What changes were proposed in this pull request?
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62593/testReport/junit/org.apache.spark.sql.sources/CreateTableAsSelectSuite/create_a_table__drop_it_and_create_another_one_with_the_same_name/ shows that `create a table, drop it and create another one with the same name` failed. But other runs were good. Seems it is a flaky test. This PR tries to make this test more stable.

Author: Yin Huai <yhuai@databricks.com>

Closes #14289 from yhuai/SPARK-16656.
2016-07-21 12:10:26 -07:00
Cheng Lian 69626adddc [SPARK-16632][SQL] Revert PR #14272: Respect Hive schema when merging parquet schema
## What changes were proposed in this pull request?

PR #14278 is a more general and simpler fix for SPARK-16632 than PR #14272. After merging #14278, we no longer need changes made in #14272. So here I revert them.

This PR targets both master and branch-2.0.

## How was this patch tested?

Existing tests.

Author: Cheng Lian <lian@databricks.com>

Closes #14300 from liancheng/revert-pr-14272.
2016-07-21 22:08:34 +08:00
Liang-Chi Hsieh 6203668d50 [SPARK-16640][SQL] Add codegen for Elt function
## What changes were proposed in this pull request?

Elt function doesn't support codegen execution now. We should add the support.

## How was this patch tested?

Jenkins tests.

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

Closes #14277 from viirya/elt-codegen.
2016-07-21 20:54:17 +08:00
Cheng Lian 8674054d34 [SPARK-16632][SQL] Use Spark requested schema to guide vectorized Parquet reader initialization
## What changes were proposed in this pull request?

In `SpecificParquetRecordReaderBase`, which is used by the vectorized Parquet reader, we convert the Parquet requested schema into a Spark schema to guide column reader initialization. However, the Parquet requested schema is tailored from the schema of the physical file being scanned, and may have inaccurate type information due to bugs of other systems (e.g. HIVE-14294).

On the other hand, we already set the real Spark requested schema into Hadoop configuration in [`ParquetFileFormat`][1]. This PR simply reads out this schema to replace the converted one.

## How was this patch tested?

New test case added in `ParquetQuerySuite`.

[1]: https://github.com/apache/spark/blob/v2.0.0-rc5/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala#L292-L294

Author: Cheng Lian <lian@databricks.com>

Closes #14278 from liancheng/spark-16632-simpler-fix.
2016-07-21 17:15:07 +08:00
Sean Owen 864b764eaf [SPARK-16226][SQL] Weaken JDBC isolation level to avoid locking when writing partitions
## What changes were proposed in this pull request?

Saving partitions to JDBC in transaction can use a weaker transaction isolation level to reduce locking. Use better method to check if transactions are supported.

## How was this patch tested?

Existing Jenkins tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #14054 from srowen/SPARK-16226.
2016-07-21 09:23:41 +01:00
Wenchen Fan cfa5ae84ed [SPARK-16644][SQL] Aggregate should not propagate constraints containing aggregate expressions
## What changes were proposed in this pull request?

aggregate expressions can only be executed inside `Aggregate`, if we propagate it up with constraints, the parent operator can not execute it and will fail at runtime.

## How was this patch tested?

new test in SQLQuerySuite

Author: Wenchen Fan <wenchen@databricks.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #14281 from cloud-fan/bug.
2016-07-20 18:37:15 -07:00
Marcelo Vanzin 75a06aa256 [SPARK-16272][CORE] Allow config values to reference conf, env, system props.
This allows configuration to be more flexible, for example, when the cluster does
not have a homogeneous configuration (e.g. packages are installed on different
paths in different nodes). By allowing one to reference the environment from
the conf, it becomes possible to work around those in certain cases.

As part of the implementation, ConfigEntry now keeps track of all "known" configs
(i.e. those created through the use of ConfigBuilder), since that list is used
by the resolution code. This duplicates some code in SQLConf, which could potentially
be merged with this now. It will also make it simpler to implement some missing
features such as filtering which configs show up in the UI or in event logs - which
are not part of this change.

Another change is in the way ConfigEntry reads config data; it now takes a string
map and a function that reads env variables, so that it can be called both from
SparkConf and SQLConf. This makes it so both places follow the same read path,
instead of having to replicate certain logic in SQLConf. There are still a
couple of methods in SQLConf that peek into fields of ConfigEntry directly,
though.

Tested via unit tests, and by using the new variable expansion functionality
in a shell session with a custom spark.sql.hive.metastore.jars value.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #14022 from vanzin/SPARK-16272.
2016-07-20 18:24:35 -07:00
Cheng Lian e651900bd5 [SPARK-16344][SQL] Decoding Parquet array of struct with a single field named "element"
## What changes were proposed in this pull request?

Due to backward-compatibility reasons, the following Parquet schema is ambiguous:

```
optional group f (LIST) {
  repeated group list {
    optional group element {
      optional int32 element;
    }
  }
}
```

According to the parquet-format spec, when interpreted as a standard 3-level layout, this type is equivalent to the following SQL type:

```
ARRAY<STRUCT<element: INT>>
```

However, when interpreted as a legacy 2-level layout, it's equivalent to

```
ARRAY<STRUCT<element: STRUCT<element: INT>>>
```

Historically, to disambiguate these cases, we employed two methods:

- `ParquetSchemaConverter.isElementType()`

  Used to disambiguate the above cases while converting Parquet types to Spark types.

- `ParquetRowConverter.isElementType()`

  Used to disambiguate the above cases while instantiating row converters that convert Parquet records to Spark rows.

Unfortunately, these two methods make different decision about the above problematic Parquet type, and caused SPARK-16344.

`ParquetRowConverter.isElementType()` is necessary for Spark 1.4 and earlier versions because Parquet requested schemata are directly converted from Spark schemata in these versions. The converted Parquet schemata may be incompatible with actual schemata of the underlying physical files when the files are written by a system/library that uses a schema conversion scheme that is different from Spark when writing Parquet LIST and MAP fields.

In Spark 1.5, Parquet requested schemata are always properly tailored from schemata of physical files to be read. Thus `ParquetRowConverter.isElementType()` is no longer necessary. This PR replaces this method with a simply yet accurate scheme: whenever an ambiguous Parquet type is hit, convert the type in question back to a Spark type using `ParquetSchemaConverter` and check whether it matches the corresponding Spark type.

## How was this patch tested?

New test cases added in `ParquetHiveCompatibilitySuite` and `ParquetQuerySuite`.

Author: Cheng Lian <lian@databricks.com>

Closes #14014 from liancheng/spark-16344-for-master-and-2.0.
2016-07-20 16:49:46 -07:00
Marcelo Vanzin e3cd5b3050 [SPARK-16634][SQL] Workaround JVM bug by moving some code out of ctor.
Some 1.7 JVMs have a bug that is triggered by certain Scala-generated
bytecode. GenericArrayData suffers from that and fails to load in certain
JVMs.

Moving the offending code out of the constructor and into a helper method
avoids the issue.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #14271 from vanzin/SPARK-16634.
2016-07-20 10:38:44 -07:00
Marcelo Vanzin 75146be6ba [SPARK-16632][SQL] Respect Hive schema when merging parquet schema.
When Hive (or at least certain versions of Hive) creates parquet files
containing tinyint or smallint columns, it stores them as int32, but
doesn't annotate the parquet field as containing the corresponding
int8 / int16 data. When Spark reads those files using the vectorized
reader, it follows the parquet schema for these fields, but when
actually reading the data it tries to use the type fetched from
the metastore, and then fails because data has been loaded into the
wrong fields in OnHeapColumnVector.

So instead of blindly trusting the parquet schema, check whether the
Catalyst-provided schema disagrees with it, and adjust the types so
that the necessary metadata is present when loading the data into
the ColumnVector instance.

Tested with unit tests and with tests that create byte / short columns
in Hive and try to read them from Spark.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #14272 from vanzin/SPARK-16632.
2016-07-20 13:00:22 +08:00
Yin Huai 2ae7b88a07 [SPARK-15705][SQL] Change the default value of spark.sql.hive.convertMetastoreOrc to false.
## What changes were proposed in this pull request?
In 2.0, we add a new logic to convert HiveTableScan on ORC tables to Spark's native code path. However, during this conversion, we drop the original metastore schema (https://issues.apache.org/jira/browse/SPARK-15705). Because of this regression, I am changing the default value of `spark.sql.hive.convertMetastoreOrc` to false.

Author: Yin Huai <yhuai@databricks.com>

Closes #14267 from yhuai/SPARK-15705-changeDefaultValue.
2016-07-19 12:58:08 -07:00
Dongjoon Hyun 162d04a30e [SPARK-16602][SQL] Nvl function should support numeric-string cases
## What changes were proposed in this pull request?

`Nvl` function should support numeric-straing cases like Hive/Spark1.6. Currently, `Nvl` finds the tightest common types among numeric types. This PR extends that to consider `String` type, too.

```scala
- TypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype =>
+ TypeCoercion.findTightestCommonTypeToString(left.dataType, right.dataType).map { dtype =>
```

**Before**
```scala
scala> sql("select nvl('0', 1)").collect()
org.apache.spark.sql.AnalysisException: cannot resolve `nvl("0", 1)` due to data type mismatch:
input to function coalesce should all be the same type, but it's [string, int]; line 1 pos 7
```

**After**
```scala
scala> sql("select nvl('0', 1)").collect()
res0: Array[org.apache.spark.sql.Row] = Array([0])
```

## How was this patch tested?

Pass the Jenkins tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14251 from dongjoon-hyun/SPARK-16602.
2016-07-19 10:28:17 -07:00
Xin Ren 21a6dd2aef [SPARK-16535][BUILD] In pom.xml, remove groupId which is redundant definition and inherited from the parent
https://issues.apache.org/jira/browse/SPARK-16535

## What changes were proposed in this pull request?

When I scan through the pom.xml of sub projects, I found this warning as below and attached screenshot
```
Definition of groupId is redundant, because it's inherited from the parent
```
![screen shot 2016-07-13 at 3 13 11 pm](https://cloud.githubusercontent.com/assets/3925641/16823121/744f893e-4916-11e6-8a52-042f83b9db4e.png)

I've tried to remove some of the lines with groupId definition, and the build on my local machine is still ok.
```
<groupId>org.apache.spark</groupId>
```
As I just find now `<maven.version>3.3.9</maven.version>` is being used in Spark 2.x, and Maven-3 supports versionless parent elements: Maven 3 will remove the need to specify the parent version in sub modules. THIS is great (in Maven 3.1).

ref: http://stackoverflow.com/questions/3157240/maven-3-worth-it/3166762#3166762

## How was this patch tested?

I've tested by re-building the project, and build succeeded.

Author: Xin Ren <iamshrek@126.com>

Closes #14189 from keypointt/SPARK-16535.
2016-07-19 11:59:46 +01:00
Reynold Xin 69c773052a [SPARK-16615][SQL] Expose sqlContext in SparkSession
## What changes were proposed in this pull request?
This patch removes the private[spark] qualifier for SparkSession.sqlContext, as discussed in http://apache-spark-developers-list.1001551.n3.nabble.com/Re-transtition-SQLContext-to-SparkSession-td18342.html

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

Author: Reynold Xin <rxin@databricks.com>

Closes #14252 from rxin/SPARK-16615.
2016-07-18 18:03:35 -07:00
Reynold Xin c4524f5193 [HOTFIX] Fix Scala 2.10 compilation 2016-07-18 17:56:36 -07:00
Dongjoon Hyun ea78edb80b [SPARK-16590][SQL] Improve LogicalPlanToSQLSuite to check generated SQL directly
## What changes were proposed in this pull request?

This PR improves `LogicalPlanToSQLSuite` to check the generated SQL directly by **structure**. So far, `LogicalPlanToSQLSuite` relies on  `checkHiveQl` to ensure the **successful SQL generation** and **answer equality**. However, it does not guarantee the generated SQL is the same or will not be changed unnoticeably.

## How was this patch tested?

Pass the Jenkins. This is only a testsuite change.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14235 from dongjoon-hyun/SPARK-16590.
2016-07-18 17:17:37 -07:00
Daoyuan Wang 96e9afaae9 [SPARK-16515][SQL] set default record reader and writer for script transformation
## What changes were proposed in this pull request?
In ScriptInputOutputSchema, we read default RecordReader and RecordWriter from conf. Since Spark 2.0 has deleted those config keys from hive conf, we have to set default reader/writer class name by ourselves. Otherwise we will get None for LazySimpleSerde, the data written would not be able to read by script. The test case added worked fine with previous version of Spark, but would fail now.

## How was this patch tested?
added a test case in SQLQuerySuite.

Closes #14169

Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #14249 from yhuai/scriptTransformation.
2016-07-18 13:58:12 -07:00
hyukjinkwon 2877f1a522 [SPARK-16351][SQL] Avoid per-record type dispatch in JSON when writing
## What changes were proposed in this pull request?

Currently, `JacksonGenerator.apply` is doing type-based dispatch for each row to write appropriate values.
It might not have to be done like this because the schema is already kept.

So, appropriate writers can be created first according to the schema once, and then apply them to each row. This approach is similar with `CatalystWriteSupport`.

This PR corrects `JacksonGenerator` so that it creates all writers for the schema once and then applies them to each row rather than type dispatching for every row.

Benchmark was proceeded with the codes below:

```scala
test("Benchmark for JSON writer") {
  val N = 500 << 8
  val row =
    """{"struct":{"field1": true, "field2": 92233720368547758070},
      "structWithArrayFields":{"field1":[4, 5, 6], "field2":["str1", "str2"]},
      "arrayOfString":["str1", "str2"],
      "arrayOfInteger":[1, 2147483647, -2147483648],
      "arrayOfLong":[21474836470, 9223372036854775807, -9223372036854775808],
      "arrayOfBigInteger":[922337203685477580700, -922337203685477580800],
      "arrayOfDouble":[1.2, 1.7976931348623157E308, 4.9E-324, 2.2250738585072014E-308],
      "arrayOfBoolean":[true, false, true],
      "arrayOfNull":[null, null, null, null],
      "arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}],
      "arrayOfArray1":[[1, 2, 3], ["str1", "str2"]],
      "arrayOfArray2":[[1, 2, 3], [1.1, 2.1, 3.1]]
     }"""
  val df = spark.sqlContext.read.json(spark.sparkContext.parallelize(List.fill(N)(row)))
  val benchmark = new Benchmark("JSON writer", N)
  benchmark.addCase("writing JSON file", 10) { _ =>
    withTempPath { path =>
      df.write.format("json").save(path.getCanonicalPath)
    }
  }
  benchmark.run()
}
```

This produced the results below

- **Before**

```
JSON writer:                             Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
writing JSON file                             1675 / 1767          0.1       13087.5       1.0X
```

- **After**

```
JSON writer:                             Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
writing JSON file                             1597 / 1686          0.1       12477.1       1.0X
```

In addition, I ran this benchmark 10 times for each and calculated the average elapsed time as below:

| **Before** | **After**|
|---------------|------------|
|17478ms  |16669ms |

It seems roughly ~5% is improved.

## How was this patch tested?

Existing tests should cover this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14028 from HyukjinKwon/SPARK-16351.
2016-07-18 09:49:14 -07:00
Reynold Xin 480c870644 [SPARK-16588][SQL] Deprecate monotonicallyIncreasingId in Scala/Java
This patch deprecates monotonicallyIncreasingId in Scala/Java, as done in Python.

This patch was originally written by HyukjinKwon. Closes #14236.
2016-07-17 22:48:00 -07:00
Reynold Xin 7b84758034 [SPARK-16584][SQL] Move regexp unit tests to RegexpExpressionsSuite
## What changes were proposed in this pull request?
This patch moves regexp related unit tests from StringExpressionsSuite to RegexpExpressionsSuite to match the file name for regexp expressions.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #14230 from rxin/SPARK-16584.
2016-07-16 23:42:28 -07:00
Sameer Agarwal a1ffbada8a [SPARK-16582][SQL] Explicitly define isNull = false for non-nullable expressions
## What changes were proposed in this pull request?

This patch is just a slightly safer way to fix the issue we encountered in https://github.com/apache/spark/pull/14168 should this pattern re-occur at other places in the code.

## How was this patch tested?

Existing tests. Also, I manually tested that it fixes the problem in SPARK-16514 without having the proposed change in https://github.com/apache/spark/pull/14168

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

Closes #14227 from sameeragarwal/codegen.
2016-07-16 13:24:00 -07:00
Reynold Xin 2e4075e2ec [SPARK-16557][SQL] Remove stale doc in sql/README.md
## What changes were proposed in this pull request?
Most of the documentation in https://github.com/apache/spark/blob/master/sql/README.md is stale. It would be useful to keep the list of projects to explain what's going on, and everything else should be removed.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #14211 from rxin/SPARK-16557.
2016-07-14 19:24:42 -07:00
Jacek Lewandowski 31ca741aef [SPARK-16528][SQL] Fix NPE problem in HiveClientImpl
## What changes were proposed in this pull request?

There are some calls to methods or fields (getParameters, properties) which are then passed to Java/Scala collection converters. Unfortunately those fields can be null in some cases and then the conversions throws NPE. We fix it by wrapping calls to those fields and methods with option and then do the conversion.

## How was this patch tested?

Manually tested with a custom Hive metastore.

Author: Jacek Lewandowski <lewandowski.jacek@gmail.com>

Closes #14200 from jacek-lewandowski/SPARK-16528.
2016-07-14 10:18:31 -07:00
Dongjoon Hyun c576f9fb90 [SPARK-16529][SQL][TEST] withTempDatabase should set default database before dropping
## What changes were proposed in this pull request?

`SQLTestUtils.withTempDatabase` is a frequently used test harness to setup a temporary table and clean up finally. This issue improves like the following for usability.

```scala
-    try f(dbName) finally spark.sql(s"DROP DATABASE $dbName CASCADE")
+    try f(dbName) finally {
+      if (spark.catalog.currentDatabase == dbName) {
+        spark.sql(s"USE ${DEFAULT_DATABASE}")
+      }
+      spark.sql(s"DROP DATABASE $dbName CASCADE")
+    }
```

In case of forgetting to reset the databaes, `withTempDatabase` will not raise Exception.

## How was this patch tested?

This improves test harness.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14184 from dongjoon-hyun/SPARK-16529.
2016-07-15 00:51:11 +08:00
Dongjoon Hyun 56183b84fb [SPARK-16543][SQL] Rename the columns of SHOW PARTITION/COLUMNS commands
## What changes were proposed in this pull request?

This PR changes the name of columns returned by `SHOW PARTITION` and `SHOW COLUMNS` commands. Currently, both commands uses `result` as a column name.

**Comparison: Column Name**

Command|Spark(Before)|Spark(After)|Hive
----------|--------------|------------|-----
SHOW PARTITIONS|result|partition|partition
SHOW COLUMNS|result|col_name|field

Note that Spark/Hive uses `col_name` in `DESC TABLES`. So, this PR chooses `col_name` for consistency among Spark commands.

**Before**
```scala
scala> sql("show partitions p").show()
+------+
|result|
+------+
|   b=2|
+------+

scala> sql("show columns in p").show()
+------+
|result|
+------+
|     a|
|     b|
+------+
```

**After**
```scala
scala> sql("show partitions p").show
+---------+
|partition|
+---------+
|      b=2|
+---------+

scala> sql("show columns in p").show
+--------+
|col_name|
+--------+
|       a|
|       b|
+--------+
```

## How was this patch tested?

Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14199 from dongjoon-hyun/SPARK-16543.
2016-07-14 17:18:34 +02:00
gatorsmile 1b5c9e52a7 [SPARK-16530][SQL][TRIVIAL] Wrong Parser Keyword in ALTER TABLE CHANGE COLUMN
#### What changes were proposed in this pull request?
Based on the [Hive SQL syntax](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-ChangeColumnName/Type/Position/Comment), the command to change column name/type/position/comment is `ALTER TABLE CHANGE COLUMN`. However, in our .g4 file, it is `ALTER TABLE CHANGE COLUMNS`. Because it is the last optional keyword, it does not take any effect. Thus, I put the issue as a Trivial level.

cc hvanhovell

#### How was this patch tested?
Existing test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14186 from gatorsmile/changeColumns.
2016-07-14 17:15:51 +02:00
Wenchen Fan db7317ac3c [SPARK-16448] RemoveAliasOnlyProject should not remove alias with metadata
## What changes were proposed in this pull request?

`Alias` with metadata is not a no-op and we should not strip it in `RemoveAliasOnlyProject` rule.
This PR also did some improvement for this rule:

1. extend the semantic of `alias-only`. Now we allow the project list to be partially aliased.
2. add unit test for this rule.

## How was this patch tested?

new `RemoveAliasOnlyProjectSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14106 from cloud-fan/bug.
2016-07-14 15:48:22 +08:00
Liwei Lin 39c836e976 [SPARK-16503] SparkSession should provide Spark version
## What changes were proposed in this pull request?

This patch enables SparkSession to provide spark version.

## How was this patch tested?

Manual test:

```
scala> sc.version
res0: String = 2.1.0-SNAPSHOT

scala> spark.version
res1: String = 2.1.0-SNAPSHOT
```

```
>>> sc.version
u'2.1.0-SNAPSHOT'
>>> spark.version
u'2.1.0-SNAPSHOT'
```

Author: Liwei Lin <lwlin7@gmail.com>

Closes #14165 from lw-lin/add-version.
2016-07-13 22:30:46 -07:00
gatorsmile c5ec879828 [SPARK-16482][SQL] Describe Table Command for Tables Requiring Runtime Inferred Schema
#### What changes were proposed in this pull request?
If we create a table pointing to a parquet/json datasets without specifying the schema, describe table command does not show the schema at all. It only shows `# Schema of this table is inferred at runtime`. In 1.6, describe table does show the schema of such a table.

~~For data source tables, to infer the schema, we need to load the data source tables at runtime. Thus, this PR calls the function `lookupRelation`.~~

For data source tables, we infer the schema before table creation. Thus, this PR set the inferred schema as the table schema when table creation.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14148 from gatorsmile/describeSchema.
2016-07-13 15:23:37 -07:00
Burak Yavuz 0744d84c91 [SPARK-16531][SQL][TEST] Remove timezone setting from DataFrameTimeWindowingSuite
## What changes were proposed in this pull request?

It's unnecessary. `QueryTest` already sets it.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #14170 from brkyvz/test-tz.
2016-07-13 12:54:57 -07:00
蒋星博 f376c37268 [SPARK-16343][SQL] Improve the PushDownPredicate rule to pushdown predicates correctly in non-deterministic condition.
## What changes were proposed in this pull request?

Currently our Optimizer may reorder the predicates to run them more efficient, but in non-deterministic condition, change the order between deterministic parts and non-deterministic parts may change the number of input rows. For example:
```SELECT a FROM t WHERE rand() < 0.1 AND a = 1```
And
```SELECT a FROM t WHERE a = 1 AND rand() < 0.1```
may call rand() for different times and therefore the output rows differ.

This PR improved this condition by checking whether the predicate is placed before any non-deterministic predicates.

## How was this patch tested?

Expanded related testcases in FilterPushdownSuite.

Author: 蒋星博 <jiangxingbo@meituan.com>

Closes #14012 from jiangxb1987/ppd.
2016-07-14 00:21:27 +08:00
Maciej Brynski 83879ebc58 [SPARK-16439] Fix number formatting in SQL UI
## What changes were proposed in this pull request?

Spark SQL UI display numbers greater than 1000 with u00A0 as grouping separator.
Problem exists when server locale has no-breaking space as separator. (for example pl_PL)
This patch turns off grouping and remove this separator.

The problem starts with this PR.
https://github.com/apache/spark/pull/12425/files#diff-803f475b01acfae1c5c96807c2ea9ddcR125

## How was this patch tested?

Manual UI tests. Screenshot attached.

![image](https://cloud.githubusercontent.com/assets/4006010/16749556/5cb5a372-47cb-11e6-9a95-67fd3f9d1c71.png)

Author: Maciej Brynski <maciej.brynski@adpilot.pl>

Closes #14142 from maver1ck/master.
2016-07-13 10:50:26 +01:00
Xin Ren f73891e0b9 [MINOR] Fix Java style errors and remove unused imports
## What changes were proposed in this pull request?

Fix Java style errors and remove unused imports, which are randomly found

## How was this patch tested?

Tested on my local machine.

Author: Xin Ren <iamshrek@126.com>

Closes #14161 from keypointt/SPARK-16437.
2016-07-13 10:47:07 +01:00
Sean Owen c190d89bd3 [SPARK-15889][STREAMING] Follow-up fix to erroneous condition in StreamTest
## What changes were proposed in this pull request?

A second form of AssertQuery now actually invokes the condition; avoids a build warning too

## How was this patch tested?

Jenkins; running StreamTest

Author: Sean Owen <sowen@cloudera.com>

Closes #14133 from srowen/SPARK-15889.2.
2016-07-13 10:44:07 +01:00
Eric Liang 1c58fa905b [SPARK-16514][SQL] Fix various regex codegen bugs
## What changes were proposed in this pull request?

RegexExtract and RegexReplace currently crash on non-nullable input due use of a hard-coded local variable name (e.g. compiles fail with `java.lang.Exception: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 85, Column 26: Redefinition of local variable "m" `).

This changes those variables to use fresh names, and also in a few other places.

## How was this patch tested?

Unit tests. rxin

Author: Eric Liang <ekl@databricks.com>

Closes #14168 from ericl/sc-3906.
2016-07-12 23:09:02 -07:00
petermaxlee 56bd399a86 [SPARK-16284][SQL] Implement reflect SQL function
## What changes were proposed in this pull request?
This patch implements reflect SQL function, which can be used to invoke a Java method in SQL. Slightly different from Hive, this implementation requires the class name and the method name to be literals. This implementation also supports only a smaller number of data types, and requires the function to be static, as suggested by rxin in #13969.

java_method is an alias for reflect, so this should also resolve SPARK-16277.

## How was this patch tested?
Added expression unit tests and an end-to-end test.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14138 from petermaxlee/reflect-static.
2016-07-13 08:05:20 +08:00
Marcelo Vanzin 7f968867ff [SPARK-16119][SQL] Support PURGE option to drop table / partition.
This option is used by Hive to directly delete the files instead of
moving them to the trash. This is needed in certain configurations
where moving the files does not work. For non-Hive tables and partitions,
Spark already behaves as if the PURGE option was set, so there's no
need to do anything.

Hive support for PURGE was added in 0.14 (for tables) and 1.2 (for
partitions), so the code reflects that: trying to use the option with
older versions of Hive will cause an exception to be thrown.

The change is a little noisier than I would like, because of the code
to propagate the new flag through all the interfaces and implementations;
the main changes are in the parser and in HiveShim, aside from the tests
(DDLCommandSuite, VersionsSuite).

Tested by running sql and catalyst unit tests, plus VersionsSuite which
has been updated to test the version-specific behavior. I also ran an
internal test suite that uses PURGE and would not pass previously.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #13831 from vanzin/SPARK-16119.
2016-07-12 12:47:46 -07:00
Reynold Xin c377e49e38 [SPARK-16489][SQL] Guard against variable reuse mistakes in expression code generation
## What changes were proposed in this pull request?
In code generation, it is incorrect for expressions to reuse variable names across different instances of itself. As an example, SPARK-16488 reports a bug in which pmod expression reuses variable name "r".

This patch updates ExpressionEvalHelper test harness to always project two instances of the same expression, which will help us catch variable reuse problems in expression unit tests. This patch also fixes the bug in crc32 expression.

## How was this patch tested?
This is a test harness change, but I also created a new test suite for testing the test harness.

Author: Reynold Xin <rxin@databricks.com>

Closes #14146 from rxin/SPARK-16489.
2016-07-12 10:07:23 -07:00
Lianhui Wang 5ad68ba5ce [SPARK-15752][SQL] Optimize metadata only query that has an aggregate whose children are deterministic project or filter operators.
## What changes were proposed in this pull request?
when query only use metadata (example: partition key), it can return results based on metadata without scanning files. Hive did it in HIVE-1003.

## How was this patch tested?
add unit tests

Author: Lianhui Wang <lianhuiwang09@gmail.com>
Author: Wenchen Fan <wenchen@databricks.com>
Author: Lianhui Wang <lianhuiwang@users.noreply.github.com>

Closes #13494 from lianhuiwang/metadata-only.
2016-07-12 18:52:15 +02:00
Takuya UESHIN 5b28e02584 [SPARK-16189][SQL] Add ExternalRDD logical plan for input with RDD to have a chance to eliminate serialize/deserialize.
## What changes were proposed in this pull request?

Currently the input `RDD` of `Dataset` is always serialized to `RDD[InternalRow]` prior to being as `Dataset`, but there is a case that we use `map` or `mapPartitions` just after converted to `Dataset`.
In this case, serialize and then deserialize happens but it would not be needed.

This pr adds `ExistingRDD` logical plan for input with `RDD` to have a chance to eliminate serialize/deserialize.

## How was this patch tested?

Existing tests.

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

Closes #13890 from ueshin/issues/SPARK-16189.
2016-07-12 17:16:59 +08:00
petermaxlee c9a6762150 [SPARK-16199][SQL] Add a method to list the referenced columns in data source Filter
## What changes were proposed in this pull request?
It would be useful to support listing the columns that are referenced by a filter. This can help simplify data source planning, because with this we would be able to implement unhandledFilters method in HadoopFsRelation.

This is based on rxin's patch (#13901) and adds unit tests.

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

Author: petermaxlee <petermaxlee@gmail.com>
Author: Reynold Xin <rxin@databricks.com>

Closes #14120 from petermaxlee/SPARK-16199.
2016-07-11 22:23:32 -07:00
Russell Spitzer b1e5281c5c [SPARK-12639][SQL] Mark Filters Fully Handled By Sources with *
## What changes were proposed in this pull request?

In order to make it clear which filters are fully handled by the
underlying datasource we will mark them with an *. This will give a
clear visual queue to users that the filter is being treated differently
by catalyst than filters which are just presented to the underlying
DataSource.

Examples from the FilteredScanSuite, in this example `c IN (...)` is handled by the source, `b < ...` is not
### Before
```
//SELECT a FROM oneToTenFiltered WHERE a + b > 9 AND b < 16 AND c IN ('bbbbbBBBBB', 'cccccCCCCC', 'dddddDDDDD', 'foo')
== Physical Plan ==
Project [a#0]
+- Filter (((a#0 + b#1) > 9) && (b#1 < 16))
   +- Scan SimpleFilteredScan(1,10)[a#0,b#1] PushedFilters: [LessThan(b,16), In(c, [bbbbbBBBBB,cccccCCCCC,dddddDDDDD,foo]]
```

### After
```
== Physical Plan ==
Project [a#0]
+- Filter (((a#0 + b#1) > 9) && (b#1 < 16))
   +- Scan SimpleFilteredScan(1,10)[a#0,b#1] PushedFilters: [LessThan(b,16), *In(c, [bbbbbBBBBB,cccccCCCCC,dddddDDDDD,foo]]
```

## How was the this patch tested?

Manually tested with the Spark Cassandra Connector, a source which fully handles underlying filters. Now fully handled filters appear with an * next to their names. I can add an automated test as well if requested

Post 1.6.1
Tested by modifying the FilteredScanSuite to run explains.

Author: Russell Spitzer <Russell.Spitzer@gmail.com>

Closes #11317 from RussellSpitzer/SPARK-12639-Star.
2016-07-11 21:40:09 -07:00
Sameer Agarwal 9cc74f95ed [SPARK-16488] Fix codegen variable namespace collision in pmod and partitionBy
## What changes were proposed in this pull request?

This patch fixes a variable namespace collision bug in pmod and partitionBy

## How was this patch tested?

Regression test for one possible occurrence. A more general fix in `ExpressionEvalHelper.checkEvaluation` will be in a subsequent PR.

Author: Sameer Agarwal <sameer@databricks.com>

Closes #14144 from sameeragarwal/codegen-bug.
2016-07-11 20:26:01 -07:00
Tathagata Das e50efd53f0 [SPARK-16430][SQL][STREAMING] Fixed bug in the maxFilesPerTrigger in FileStreamSource
## What changes were proposed in this pull request?

Incorrect list of files were being allocated to a batch. This caused a file to read multiple times in the multiple batches.

## How was this patch tested?

Added unit tests

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

Closes #14143 from tdas/SPARK-16430-1.
2016-07-11 18:41:36 -07:00
Shixiong Zhu 91a443b849 [SPARK-16433][SQL] Improve StreamingQuery.explain when no data arrives
## What changes were proposed in this pull request?

Display `No physical plan. Waiting for data.` instead of `N/A`  for StreamingQuery.explain when no data arrives because `N/A` doesn't provide meaningful information.

## How was this patch tested?

Existing unit tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #14100 from zsxwing/SPARK-16433.
2016-07-11 18:11:06 -07:00
James Thomas 9e2c763dbb [SPARK-16114][SQL] structured streaming event time window example
## What changes were proposed in this pull request?

A structured streaming example with event time windowing.

## How was this patch tested?

Run locally

Author: James Thomas <jamesjoethomas@gmail.com>

Closes #13957 from jjthomas/current.
2016-07-11 17:57:51 -07:00
Marcelo Vanzin b4fbe140be [SPARK-16349][SQL] Fall back to isolated class loader when classes not found.
Some Hadoop classes needed by the Hive metastore client jars are not present
in Spark's packaging (for example, "org/apache/hadoop/mapred/MRVersion"). So
if the parent class loader fails to find a class, try to load it from the
isolated class loader, in case it's available there.

Tested by setting spark.sql.hive.metastore.jars to local paths with Hive/Hadoop
libraries and verifying that Spark can talk to the metastore.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #14020 from vanzin/SPARK-16349.
2016-07-11 15:20:48 -07:00
Dongjoon Hyun 840853ed06 [SPARK-16458][SQL] SessionCatalog should support listColumns for temporary tables
## What changes were proposed in this pull request?

Temporary tables are used frequently, but `spark.catalog.listColumns` does not support those tables. This PR make `SessionCatalog` supports temporary table column listing.

**Before**
```scala
scala> spark.range(10).createOrReplaceTempView("t1")

scala> spark.catalog.listTables().collect()
res1: Array[org.apache.spark.sql.catalog.Table] = Array(Table[name=`t1`, tableType=`TEMPORARY`, isTemporary=`true`])

scala> spark.catalog.listColumns("t1").collect()
org.apache.spark.sql.AnalysisException: Table `t1` does not exist in database `default`.;
```

**After**
```
scala> spark.catalog.listColumns("t1").collect()
res2: Array[org.apache.spark.sql.catalog.Column] = Array(Column[name='id', description='id', dataType='bigint', nullable='false', isPartition='false', isBucket='false'])
```
## How was this patch tested?

Pass the Jenkins tests including a new testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14114 from dongjoon-hyun/SPARK-16458.
2016-07-11 22:45:22 +02:00
Reynold Xin ffcb6e055a [SPARK-16477] Bump master version to 2.1.0-SNAPSHOT
## What changes were proposed in this pull request?
After SPARK-16476 (committed earlier today as #14128), we can finally bump the version number.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #14130 from rxin/SPARK-16477.
2016-07-11 09:42:56 -07:00
Dongjoon Hyun 7ac79da0e4 [SPARK-16459][SQL] Prevent dropping current database
## What changes were proposed in this pull request?

This PR prevents dropping the current database to avoid errors like the followings.

```scala
scala> sql("create database delete_db")
scala> sql("use delete_db")
scala> sql("drop database delete_db")
scala> sql("create table t as select 1")
org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException: Database `delete_db` not found;
```

## How was this patch tested?

Pass the Jenkins tests including an updated testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14115 from dongjoon-hyun/SPARK-16459.
2016-07-11 15:15:47 +02:00
gatorsmile e226278941 [SPARK-16355][SPARK-16354][SQL] Fix Bugs When LIMIT/TABLESAMPLE is Non-foldable, Zero or Negative
#### What changes were proposed in this pull request?
**Issue 1:** When a query containing LIMIT/TABLESAMPLE 0, the statistics could be zero. Results are correct but it could cause a huge performance regression. For example,
```Scala
Seq(("one", 1), ("two", 2), ("three", 3), ("four", 4)).toDF("k", "v")
  .createOrReplaceTempView("test")
val df1 = spark.table("test")
val df2 = spark.table("test").limit(0)
val df = df1.join(df2, Seq("k"), "left")
```
The statistics of both `df` and `df2` are zero. The statistics values should never be zero; otherwise `sizeInBytes` of `BinaryNode` will also be zero (product of children). This PR is to increase it to `1` when the num of rows is equal to 0.

**Issue 2:** When a query containing negative LIMIT/TABLESAMPLE, we should issue exceptions. Negative values could break the implementation assumption of multiple parts. For example, statistics calculation.  Below is the example query.
```SQL
SELECT * FROM testData TABLESAMPLE (-1 rows)
SELECT * FROM testData LIMIT -1
```
This PR is to issue an appropriate exception in this case.

**Issue 3:** Spark SQL follows the restriction of LIMIT clause in Hive. The argument to the LIMIT clause must evaluate to a constant value. It can be a numeric literal, or another kind of numeric expression involving operators, casts, and function return values. You cannot refer to a column or use a subquery. Currently, we do not detect whether the expression in LIMIT clause is foldable or not. If non-foldable, we might issue a strange error message. For example,
```SQL
SELECT * FROM testData LIMIT rand() > 0.2
```
Then, a misleading error message is issued, like
```
assertion failed: No plan for GlobalLimit (_nondeterministic#203 > 0.2)
+- Project [key#11, value#12, rand(-1441968339187861415) AS _nondeterministic#203]
   +- LocalLimit (_nondeterministic#202 > 0.2)
      +- Project [key#11, value#12, rand(-1308350387169017676) AS _nondeterministic#202]
         +- LogicalRDD [key#11, value#12]

java.lang.AssertionError: assertion failed: No plan for GlobalLimit (_nondeterministic#203 > 0.2)
+- Project [key#11, value#12, rand(-1441968339187861415) AS _nondeterministic#203]
   +- LocalLimit (_nondeterministic#202 > 0.2)
      +- Project [key#11, value#12, rand(-1308350387169017676) AS _nondeterministic#202]
         +- LogicalRDD [key#11, value#12]
```
This PR detects it and then issues a meaningful error message.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14034 from gatorsmile/limit.
2016-07-11 16:21:13 +08:00
petermaxlee 82f0874453 [SPARK-16318][SQL] Implement all remaining xpath functions
## What changes were proposed in this pull request?
This patch implements all remaining xpath functions that Hive supports and not natively supported in Spark: xpath_int, xpath_short, xpath_long, xpath_float, xpath_double, xpath_string, and xpath.

## How was this patch tested?
Added unit tests and end-to-end tests.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #13991 from petermaxlee/SPARK-16318.
2016-07-11 13:28:34 +08:00
gatorsmile 7374e518e2 [SPARK-16401][SQL] Data Source API: Enable Extending RelationProvider and CreatableRelationProvider without Extending SchemaRelationProvider
#### What changes were proposed in this pull request?
When users try to implement a data source API with extending only `RelationProvider` and `CreatableRelationProvider`, they will hit an error when resolving the relation.
```Scala
spark.read
.format("org.apache.spark.sql.test.DefaultSourceWithoutUserSpecifiedSchema")
  .load()
  .write.
format("org.apache.spark.sql.test.DefaultSourceWithoutUserSpecifiedSchema")
  .save()
```

The error they hit is like
```
org.apache.spark.sql.test.DefaultSourceWithoutUserSpecifiedSchema does not allow user-specified schemas.;
org.apache.spark.sql.AnalysisException: org.apache.spark.sql.test.DefaultSourceWithoutUserSpecifiedSchema does not allow user-specified schemas.;
	at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:319)
	at org.apache.spark.sql.execution.datasources.DataSource.write(DataSource.scala:494)
	at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:211)
```

Actually, the bug fix is simple. [`DataSource.createRelation(sparkSession.sqlContext, mode, options, data)`](dd644f8117/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala (L429)) already returns a BaseRelation. We should not assign schema to `userSpecifiedSchema`. That schema assignment only makes sense for the data sources that extend `FileFormat`.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14075 from gatorsmile/dataSource.
2016-07-09 20:35:45 +08:00
Dongjoon Hyun 3b22291b5f [SPARK-16387][SQL] JDBC Writer should use dialect to quote field names.
## What changes were proposed in this pull request?

Currently, JDBC Writer uses dialects to get datatypes, but doesn't to quote field names. This PR uses dialects to quote the field names, too.

**Reported Error Scenario (MySQL case)**
```scala
scala> val url="jdbc:mysql://localhost:3306/temp"
scala> val prop = new java.util.Properties
scala> prop.setProperty("user","root")
scala> spark.createDataset(Seq("a","b","c")).toDF("order")
scala> df.write.mode("overwrite").jdbc(url, "temptable", prop)
...MySQLSyntaxErrorException: ... near 'order TEXT )
```

## How was this patch tested?

Pass the Jenkins tests and manually do the above case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14107 from dongjoon-hyun/SPARK-16387.
2016-07-08 16:07:12 -07:00
wujian f5fef69143 [SPARK-16281][SQL] Implement parse_url SQL function
## What changes were proposed in this pull request?

This PR adds parse_url SQL functions in order to remove Hive fallback.

A new implementation of #13999

## How was this patch tested?

Pass the exist tests including new testcases.

Author: wujian <jan.chou.wu@gmail.com>

Closes #14008 from janplus/SPARK-16281.
2016-07-08 14:38:05 -07:00
Dongjoon Hyun 142df4834b [SPARK-16429][SQL] Include StringType columns in describe()
## What changes were proposed in this pull request?

Currently, Spark `describe` supports `StringType`. However, `describe()` returns a dataset for only all numeric columns. This PR aims to include `StringType` columns in `describe()`, `describe` without argument.

**Background**
```scala
scala> spark.read.json("examples/src/main/resources/people.json").describe("age", "name").show()
+-------+------------------+-------+
|summary|               age|   name|
+-------+------------------+-------+
|  count|                 2|      3|
|   mean|              24.5|   null|
| stddev|7.7781745930520225|   null|
|    min|                19|   Andy|
|    max|                30|Michael|
+-------+------------------+-------+
```

**Before**
```scala
scala> spark.read.json("examples/src/main/resources/people.json").describe().show()
+-------+------------------+
|summary|               age|
+-------+------------------+
|  count|                 2|
|   mean|              24.5|
| stddev|7.7781745930520225|
|    min|                19|
|    max|                30|
+-------+------------------+
```

**After**
```scala
scala> spark.read.json("examples/src/main/resources/people.json").describe().show()
+-------+------------------+-------+
|summary|               age|   name|
+-------+------------------+-------+
|  count|                 2|      3|
|   mean|              24.5|   null|
| stddev|7.7781745930520225|   null|
|    min|                19|   Andy|
|    max|                30|Michael|
+-------+------------------+-------+
```

## How was this patch tested?

Pass the Jenkins with a update testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14095 from dongjoon-hyun/SPARK-16429.
2016-07-08 14:36:50 -07:00
Jurriaan Pruis 38cf8f2a50 [SPARK-13638][SQL] Add quoteAll option to CSV DataFrameWriter
## What changes were proposed in this pull request?

Adds an quoteAll option for writing CSV which will quote all fields.
See https://issues.apache.org/jira/browse/SPARK-13638

## How was this patch tested?

Added a test to verify the output columns are quoted for all fields in the Dataframe

Author: Jurriaan Pruis <email@jurriaanpruis.nl>

Closes #13374 from jurriaan/csv-quote-all.
2016-07-08 11:45:41 -07:00
Dongjoon Hyun a54438cb23 [SPARK-16285][SQL] Implement sentences SQL functions
## What changes were proposed in this pull request?

This PR implements `sentences` SQL function.

## How was this patch tested?

Pass the Jenkins tests with a new testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14004 from dongjoon-hyun/SPARK_16285.
2016-07-08 17:05:24 +08:00
petermaxlee 8228b06303 [SPARK-16436][SQL] checkEvaluation should support NaN
## What changes were proposed in this pull request?
This small patch modifies ExpressionEvalHelper. checkEvaluation to support comparing NaN values for floating point comparisons.

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

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14103 from petermaxlee/SPARK-16436.
2016-07-08 16:49:02 +08:00
Dongjoon Hyun dff73bfa5e [SPARK-16052][SQL] Improve CollapseRepartition optimizer for Repartition/RepartitionBy
## What changes were proposed in this pull request?

This PR improves `CollapseRepartition` to optimize the adjacent combinations of **Repartition** and **RepartitionBy**. Also, this PR adds a testsuite for this optimizer.

**Target Scenario**
```scala
scala> val dsView1 = spark.range(8).repartition(8, $"id")
scala> dsView1.createOrReplaceTempView("dsView1")
scala> sql("select id from dsView1 distribute by id").explain(true)
```

**Before**
```scala
scala> sql("select id from dsView1 distribute by id").explain(true)
== Parsed Logical Plan ==
'RepartitionByExpression ['id]
+- 'Project ['id]
   +- 'UnresolvedRelation `dsView1`

== Analyzed Logical Plan ==
id: bigint
RepartitionByExpression [id#0L]
+- Project [id#0L]
   +- SubqueryAlias dsview1
      +- RepartitionByExpression [id#0L], 8
         +- Range (0, 8, splits=8)

== Optimized Logical Plan ==
RepartitionByExpression [id#0L]
+- RepartitionByExpression [id#0L], 8
   +- Range (0, 8, splits=8)

== Physical Plan ==
Exchange hashpartitioning(id#0L, 200)
+- Exchange hashpartitioning(id#0L, 8)
   +- *Range (0, 8, splits=8)
```

**After**
```scala
scala> sql("select id from dsView1 distribute by id").explain(true)
== Parsed Logical Plan ==
'RepartitionByExpression ['id]
+- 'Project ['id]
   +- 'UnresolvedRelation `dsView1`

== Analyzed Logical Plan ==
id: bigint
RepartitionByExpression [id#0L]
+- Project [id#0L]
   +- SubqueryAlias dsview1
      +- RepartitionByExpression [id#0L], 8
         +- Range (0, 8, splits=8)

== Optimized Logical Plan ==
RepartitionByExpression [id#0L]
+- Range (0, 8, splits=8)

== Physical Plan ==
Exchange hashpartitioning(id#0L, 200)
+- *Range (0, 8, splits=8)
```

## How was this patch tested?

Pass the Jenkins tests (including a new testsuite).

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13765 from dongjoon-hyun/SPARK-16052.
2016-07-08 16:44:53 +08:00
Tathagata Das 5bce458093 [SPARK-16430][SQL][STREAMING] Add option maxFilesPerTrigger
## What changes were proposed in this pull request?

An option that limits the file stream source to read 1 file at a time enables rate limiting. It has the additional convenience that a static set of files can be used like a stream for testing as this will allows those files to be considered one at a time.

This PR adds option `maxFilesPerTrigger`.

## How was this patch tested?

New unit test

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

Closes #14094 from tdas/SPARK-16430.
2016-07-07 23:19:41 -07:00
Daoyuan Wang 28710b42b0 [SPARK-16415][SQL] fix catalog string error
## What changes were proposed in this pull request?

In #13537 we truncate `simpleString` if it is a long `StructType`. But sometimes we need `catalogString` to reconstruct `TypeInfo`, for example in description of [SPARK-16415 ](https://issues.apache.org/jira/browse/SPARK-16415). So we need to keep the implementation of `catalogString` not affected by our truncate.

## How was this patch tested?

added a test case.

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

Closes #14089 from adrian-wang/catalogstring.
2016-07-07 11:08:06 -07:00
Liwei Lin 0f7175def9 [SPARK-16350][SQL] Fix support for incremental planning in wirteStream.foreach()
## What changes were proposed in this pull request?

There are cases where `complete` output mode does not output updated aggregated value; for details please refer to [SPARK-16350](https://issues.apache.org/jira/browse/SPARK-16350).

The cause is that, as we do `data.as[T].foreachPartition { iter => ... }` in `ForeachSink.addBatch()`, `foreachPartition()` does not support incremental planning for now.

This patches makes `foreachPartition()` support incremental planning in `ForeachSink`, by making a special version of `Dataset` with its `rdd()` method supporting incremental planning.

## How was this patch tested?

Added a unit test which failed before the change

Author: Liwei Lin <lwlin7@gmail.com>

Closes #14030 from lw-lin/fix-foreach-complete.
2016-07-07 10:40:42 -07:00
Dongjoon Hyun a04cab8f17 [SPARK-16174][SQL] Improve OptimizeIn optimizer to remove literal repetitions
## What changes were proposed in this pull request?

This PR improves `OptimizeIn` optimizer to remove the literal repetitions from SQL `IN` predicates. This optimizer prevents user mistakes and also can optimize some queries like [TPCDS-36](https://github.com/apache/spark/blob/master/sql/core/src/test/resources/tpcds/q36.sql#L19).

**Before**
```scala
scala> sql("select state from (select explode(array('CA','TN')) state) where state in ('TN','TN','TN','TN','TN','TN','TN')").explain
== Physical Plan ==
*Filter state#6 IN (TN,TN,TN,TN,TN,TN,TN)
+- Generate explode([CA,TN]), false, false, [state#6]
   +- Scan OneRowRelation[]
```

**After**
```scala
scala> sql("select state from (select explode(array('CA','TN')) state) where state in ('TN','TN','TN','TN','TN','TN','TN')").explain
== Physical Plan ==
*Filter state#6 IN (TN)
+- Generate explode([CA,TN]), false, false, [state#6]
   +- Scan OneRowRelation[]
```

## How was this patch tested?

Pass the Jenkins tests (including a new testcase).

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13876 from dongjoon-hyun/SPARK-16174.
2016-07-07 19:45:43 +08:00
Reynold Xin 986b251401 [SPARK-16400][SQL] Remove InSet filter pushdown from Parquet
## What changes were proposed in this pull request?
This patch removes InSet filter pushdown from Parquet data source, since row-based pushdown is not beneficial to Spark and brings extra complexity to the code base.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #14076 from rxin/SPARK-16400.
2016-07-07 18:09:18 +08:00
gatorsmile ab05db0b48 [SPARK-16368][SQL] Fix Strange Errors When Creating View With Unmatched Column Num
#### What changes were proposed in this pull request?
When creating a view, a common user error is the number of columns produced by the `SELECT` clause does not match the number of column names specified by `CREATE VIEW`.

For example, given Table `t1` only has 3 columns
```SQL
create view v1(col2, col4, col3, col5) as select * from t1
```
Currently, Spark SQL reports the following error:
```
requirement failed
java.lang.IllegalArgumentException: requirement failed
	at scala.Predef$.require(Predef.scala:212)
	at org.apache.spark.sql.execution.command.CreateViewCommand.run(views.scala:90)
```

This error message is very confusing. This PR is to detect the error and issue a meaningful error message.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14047 from gatorsmile/viewMismatchedColumns.
2016-07-07 00:07:25 -07:00
gatorsmile 42279bff68 [SPARK-16374][SQL] Remove Alias from MetastoreRelation and SimpleCatalogRelation
#### What changes were proposed in this pull request?
Different from the other leaf nodes, `MetastoreRelation` and `SimpleCatalogRelation` have a pre-defined `alias`, which is used to change the qualifier of the node. However, based on the existing alias handling, alias should be put in `SubqueryAlias`.

This PR is to separate alias handling from `MetastoreRelation` and `SimpleCatalogRelation` to make it consistent with the other nodes. It simplifies the signature and conversion to a `BaseRelation`.

For example, below is an example query for `MetastoreRelation`,  which is converted to a `LogicalRelation`:
```SQL
SELECT tmp.a + 1 FROM test_parquet_ctas tmp WHERE tmp.a > 2
```

Before changes, the analyzed plan is
```
== Analyzed Logical Plan ==
(a + 1): int
Project [(a#951 + 1) AS (a + 1)#952]
+- Filter (a#951 > 2)
   +- SubqueryAlias tmp
      +- Relation[a#951] parquet
```
After changes, the analyzed plan becomes
```
== Analyzed Logical Plan ==
(a + 1): int
Project [(a#951 + 1) AS (a + 1)#952]
+- Filter (a#951 > 2)
   +- SubqueryAlias tmp
      +- SubqueryAlias test_parquet_ctas
         +- Relation[a#951] parquet
```

**Note: the optimized plans are the same.**

For `SimpleCatalogRelation`, the existing code always generates two Subqueries. Thus, no change is needed.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14053 from gatorsmile/removeAliasFromMetastoreRelation.
2016-07-07 12:07:19 +08:00
hyukjinkwon 34283de160 [SPARK-14839][SQL] Support for other types for tableProperty rule in SQL syntax
## What changes were proposed in this pull request?

Currently, Scala API supports to take options with the types, `String`, `Long`, `Double` and `Boolean` and Python API also supports other types.

This PR corrects `tableProperty` rule to support other types (string, boolean, double and integer) so that support the options for data sources in a consistent way. This will affect other rules such as DBPROPERTIES and TBLPROPERTIES (allowing other types as values).

Also, `TODO add bucketing and partitioning.` was removed because it was resolved in 24bea00047

## How was this patch tested?

Unit test in `MetastoreDataSourcesSuite.scala`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #13517 from HyukjinKwon/SPARK-14839.
2016-07-06 23:57:18 -04:00
Reynold Xin 8e3e4ed6c0 [SPARK-16371][SQL] Two follow-up tasks
## What changes were proposed in this pull request?
This is a small follow-up for SPARK-16371:

1. Hide removeMetadata from public API.
2. Add JIRA ticket number to test case name.

## How was this patch tested?
Updated a test comment.

Author: Reynold Xin <rxin@databricks.com>

Closes #14074 from rxin/parquet-filter.
2016-07-06 15:04:37 -07:00
hyukjinkwon 4f8ceed593 [SPARK-16371][SQL] Do not push down filters incorrectly when inner name and outer name are the same in Parquet
## What changes were proposed in this pull request?

Currently, if there is a schema as below:

```
root
  |-- _1: struct (nullable = true)
  |    |-- _1: integer (nullable = true)
```

and if we execute the codes below:

```scala
df.filter("_1 IS NOT NULL").count()
```

This pushes down a filter although this filter is being applied to `StructType`.(If my understanding is correct, Spark does not pushes down filters for those).

The reason is, `ParquetFilters.getFieldMap` produces results below:

```
(_1,StructType(StructField(_1,IntegerType,true)))
(_1,IntegerType)
```

and then it becomes a `Map`

```
(_1,IntegerType)
```

Now, because of ` ....lift(dataTypeOf(name)).map(_(name, value))`, this pushes down filters for `_1` which Parquet thinks is `IntegerType`. However, it is actually `StructType`.

So, Parquet filter2 produces incorrect results, for example, the codes below:

```
df.filter("_1 IS NOT NULL").count()
```

produces always 0.

This PR prevents this by not finding nested fields.

## How was this patch tested?

Unit test in `ParquetFilterSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14067 from HyukjinKwon/SPARK-16371.
2016-07-06 12:42:16 -07:00
Cheng Lian 23eff5e512 [SPARK-15979][SQL] Renames CatalystWriteSupport to ParquetWriteSupport
## What changes were proposed in this pull request?

PR #13696 renamed various Parquet support classes but left `CatalystWriteSupport` behind. This PR is renames it as a follow-up.

## How was this patch tested?

N/A.

Author: Cheng Lian <lian@databricks.com>

Closes #14070 from liancheng/spark-15979-follow-up.
2016-07-06 10:36:45 -07:00
gatorsmile 21eadd1d8c [SPARK-16229][SQL] Drop Empty Table After CREATE TABLE AS SELECT fails
#### What changes were proposed in this pull request?
In `CREATE TABLE AS SELECT`, if the `SELECT` query failed, the table should not exist. For example,

```SQL
CREATE TABLE tab
STORED AS TEXTFILE
SELECT 1 AS a, (SELECT a FROM (SELECT 1 AS a UNION ALL SELECT 2 AS a) t) AS b
```
The above query failed as expected but an empty table `t` is created.

This PR is to drop the created table when hitting any non-fatal exception.

#### How was this patch tested?
Added a test case to verify the behavior

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13926 from gatorsmile/dropTableAfterException.
2016-07-06 21:43:55 +08:00
Reynold Xin 7e28fabdff [SPARK-16388][SQL] Remove spark.sql.nativeView and spark.sql.nativeView.canonical config
## What changes were proposed in this pull request?
These two configs should always be true after Spark 2.0. This patch removes them from the config list. Note that ideally this should've gone into branch-2.0, but due to the timing of the release we should only merge this in master for Spark 2.1.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #14061 from rxin/SPARK-16388.
2016-07-06 17:40:55 +08:00
Tejas Patil 5f342049cc [SPARK-16339][CORE] ScriptTransform does not print stderr when outstream is lost
## What changes were proposed in this pull request?

Currently, if due to some failure, the outstream gets destroyed or closed and later `outstream.close()` leads to IOException in such case. Due to this, the `stderrBuffer` does not get logged and there is no way for users to see why the job failed.

The change is to first display the stderr buffer and then try closing the outstream.

## How was this patch tested?

The correct way to test this fix would be to grep the log to see if the `stderrBuffer` gets logged but I dont think having test cases which do that is a good idea.

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

…

Author: Tejas Patil <tejasp@fb.com>

Closes #13834 from tejasapatil/script_transform.
2016-07-06 09:18:04 +01:00
Dongjoon Hyun ec79183ac5 [SPARK-16340][SQL] Support column arguments for regexp_replace Dataset operation
## What changes were proposed in this pull request?

Currently, `regexp_replace` function supports `Column` arguments in a query. This PR supports that in a `Dataset` operation, too.

## How was this patch tested?

Pass the Jenkins tests with a updated testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14060 from dongjoon-hyun/SPARK-16340.
2016-07-05 22:11:40 -07:00
gatorsmile ec18cd0af4 [SPARK-16389][SQL] Remove MetastoreRelation from SparkHiveWriterContainer and SparkHiveDynamicPartitionWriterContainer
#### What changes were proposed in this pull request?
- Remove useless `MetastoreRelation` from the signature of `SparkHiveWriterContainer` and `SparkHiveDynamicPartitionWriterContainer`.
- Avoid unnecessary metadata retrieval using Hive client in `InsertIntoHiveTable`.

#### How was this patch tested?
Existing test cases already cover it.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14062 from gatorsmile/removeMetastoreRelation.
2016-07-06 12:09:53 +08:00
Dongjoon Hyun d0d28507ca [SPARK-16286][SQL] Implement stack table generating function
## What changes were proposed in this pull request?

This PR implements `stack` table generating function.

## How was this patch tested?

Pass the Jenkins tests including new testcases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14033 from dongjoon-hyun/SPARK-16286.
2016-07-06 10:54:43 +08:00
Dongjoon Hyun 4db63fd2b4 [SPARK-16383][SQL] Remove SessionState.executeSql
## What changes were proposed in this pull request?

This PR removes `SessionState.executeSql` in favor of `SparkSession.sql`. We can remove this safely since the visibility `SessionState` is `private[sql]` and `executeSql` is only used in one **ignored** test, `test("Multiple Hive Instances")`.

## How was this patch tested?

Pass the Jenkins tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14055 from dongjoon-hyun/SPARK-16383.
2016-07-05 16:47:32 -07:00
Cheng Hao 920cb5fe4e [SPARK-15730][SQL] Respect the --hiveconf in the spark-sql command line
## What changes were proposed in this pull request?
This PR makes spark-sql (backed by SparkSQLCLIDriver) respects confs set by hiveconf, which is what we do in previous versions. The change is that when we start SparkSQLCLIDriver, we explicitly set confs set through --hiveconf to SQLContext's conf (basically treating those confs as a SparkSQL conf).

## How was this patch tested?
A new test in CliSuite.

Closes #13542

Author: Cheng Hao <hao.cheng@intel.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #14058 from yhuai/hiveConfThriftServer.
2016-07-05 16:42:43 -07:00
Reynold Xin 5b7a1770ac [HOTFIX] Fix build break. 2016-07-05 12:06:41 -07:00
Reynold Xin 16a2a7d714 [SPARK-16311][SQL] Metadata refresh should work on temporary views
## What changes were proposed in this pull request?
This patch fixes the bug that the refresh command does not work on temporary views. This patch is based on https://github.com/apache/spark/pull/13989, but removes the public Dataset.refresh() API as well as improved test coverage.

Note that I actually think the public refresh() API is very useful. We can in the future implement it by also invalidating the lazy vals in QueryExecution (or alternatively just create a new QueryExecution).

## How was this patch tested?
Re-enabled a previously ignored test, and added a new test suite for Hive testing behavior of temporary views against MetastoreRelation.

Author: Reynold Xin <rxin@databricks.com>
Author: petermaxlee <petermaxlee@gmail.com>

Closes #14009 from rxin/SPARK-16311.
2016-07-05 11:36:05 -07:00
hyukjinkwon 07d9c5327f [SPARK-9876][SQL][FOLLOWUP] Enable string and binary tests for Parquet predicate pushdown and replace deprecated fromByteArray.
## What changes were proposed in this pull request?

It seems Parquet has been upgraded to 1.8.1 by https://github.com/apache/spark/pull/13280. So,  this PR enables string and binary predicate push down which was disabled due to [SPARK-11153](https://issues.apache.org/jira/browse/SPARK-11153) and [PARQUET-251](https://issues.apache.org/jira/browse/PARQUET-251) and cleans up some comments unremoved (I think by mistake).

This PR also replace the API, `fromByteArray()` deprecated in [PARQUET-251](https://issues.apache.org/jira/browse/PARQUET-251).

## How was this patch tested?

Unit tests in `ParquetFilters`

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #13389 from HyukjinKwon/parquet-1.8-followup.
2016-07-05 16:59:40 +08:00
Dongjoon Hyun 7f7eb3934e [SPARK-16360][SQL] Speed up SQL query performance by removing redundant executePlan call
## What changes were proposed in this pull request?

Currently, there are a few reports about Spark 2.0 query performance regression for large queries.

This PR speeds up SQL query processing performance by removing redundant **consecutive `executePlan`** call in `Dataset.ofRows` function and `Dataset` instantiation. Specifically, this PR aims to reduce the overhead of SQL query execution plan generation, not real query execution. So, we can not see the result in the Spark Web UI. Please use the following query script. The result is **25.78 sec** -> **12.36 sec** as expected.

**Sample Query**
```scala
val n = 4000
val values = (1 to n).map(_.toString).mkString(", ")
val columns = (1 to n).map("column" + _).mkString(", ")
val query =
  s"""
     |SELECT $columns
     |FROM VALUES ($values) T($columns)
     |WHERE 1=2 AND 1 IN ($columns)
     |GROUP BY $columns
     |ORDER BY $columns
     |""".stripMargin

def time[R](block: => R): R = {
  val t0 = System.nanoTime()
  val result = block
  println("Elapsed time: " + ((System.nanoTime - t0) / 1e9) + "s")
  result
}
```

**Before**
```scala
scala> time(sql(query))
Elapsed time: 30.138142577s  // First query has a little overhead of initialization.
res0: org.apache.spark.sql.DataFrame = [column1: int, column2: int ... 3998 more fields]
scala> time(sql(query))
Elapsed time: 25.787751452s  // Let's compare this one.
res1: org.apache.spark.sql.DataFrame = [column1: int, column2: int ... 3998 more fields]
```

**After**
```scala
scala> time(sql(query))
Elapsed time: 17.500279659s  // First query has a little overhead of initialization.
res0: org.apache.spark.sql.DataFrame = [column1: int, column2: int ... 3998 more fields]
scala> time(sql(query))
Elapsed time: 12.364812255s  // This shows the real difference. The speed up is about 2 times.
res1: org.apache.spark.sql.DataFrame = [column1: int, column2: int ... 3998 more fields]
```

## How was this patch tested?

Manual by the above script.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14044 from dongjoon-hyun/SPARK-16360.
2016-07-05 16:19:22 +08:00
hyukjinkwon 7742d9f158 [SPARK-15198][SQL] Support for pushing down filters for boolean types in ORC data source
## What changes were proposed in this pull request?

It seems ORC supports all the types in  ([`PredicateLeaf.Type`](e085b7e9bd/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/PredicateLeaf.java (L50-L56))) which includes boolean types. So, this was tested first.

This PR adds the support for pushing filters down for `BooleanType` in ORC data source.

This PR also removes `OrcTableScan` class and the companion object, which is not used anymore.

## How was this patch tested?

Unittest in `OrcFilterSuite` and `OrcQuerySuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #12972 from HyukjinKwon/SPARK-15198.
2016-07-05 13:59:13 +08:00
Michael Allman 8f6cf00c69 [SPARK-15968][SQL] Nonempty partitioned metastore tables are not cached
(Please note this is a revision of PR #13686, which has been closed in favor of this PR.)

This PR addresses [SPARK-15968](https://issues.apache.org/jira/browse/SPARK-15968).

## What changes were proposed in this pull request?

The `getCached` method of [HiveMetastoreCatalog](https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala) computes `pathsInMetastore` from the metastore relation's catalog table. This only returns the table base path, which is incomplete/inaccurate for a nonempty partitioned table. As a result, cached lookups on nonempty partitioned tables always miss.

Rather than get `pathsInMetastore` from

    metastoreRelation.catalogTable.storage.locationUri.toSeq

I modified the `getCached` method to take a `pathsInMetastore` argument. Calls to this method pass in the paths computed from calls to the Hive metastore. This is how `getCached` was implemented in Spark 1.5:

e0c3212a9b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala (L444).

I also added a call in `InsertIntoHiveTable.scala` to invalidate the table from the SQL session catalog.

## How was this patch tested?

I've added a new unit test to `parquetSuites.scala`:

    SPARK-15968: nonempty partitioned metastore Parquet table lookup should use cached relation

Note that the only difference between this new test and the one above it in the file is that the new test populates its partitioned table with a single value, while the existing test leaves the table empty. This reveals a subtle, unexpected hole in test coverage present before this patch.

Note I also modified a different but related unit test in `parquetSuites.scala`:

    SPARK-15248: explicitly added partitions should be readable

This unit test asserts that Spark SQL should return data from a table partition which has been placed there outside a metastore query immediately after it is added. I changed the test so that, instead of adding the data as a parquet file saved in the partition's location, the data is added through a SQL `INSERT` query. I made this change because I could find no way to efficiently support partitioned table caching without failing that test.

In addition to my primary motivation, I can offer a few reasons I believe this is an acceptable weakening of that test. First, it still validates a fix for [SPARK-15248](https://issues.apache.org/jira/browse/SPARK-15248), the issue for which it was written. Second, the assertion made is stronger than that required for non-partitioned tables. If you write data to the storage location of a non-partitioned metastore table without using a proper SQL DML query, a subsequent call to show that data will not return it. I believe this is an intentional limitation put in place to make table caching feasible, but I'm only speculating.

Building a large `HadoopFsRelation` requires `stat`-ing all of its data files. In our environment, where we have tables with 10's of thousands of partitions, the difference between using a cached relation versus a new one is a matter of seconds versus minutes. Caching partitioned table metadata vastly improves the usability of Spark SQL for these cases.

Thanks.

Author: Michael Allman <michael@videoamp.com>

Closes #13818 from mallman/spark-15968.
2016-07-05 09:49:25 +08:00
gatorsmile 2628333978 [SPARK-16358][SQL] Remove InsertIntoHiveTable From Logical Plan
#### What changes were proposed in this pull request?
LogicalPlan `InsertIntoHiveTable` is useless. Thus, we can remove it from the code base.

#### How was this patch tested?
The existing test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14037 from gatorsmile/InsertIntoHiveTable.
2016-07-04 13:45:07 +08:00
Koert Kuipers 8cdb81fa82 [SPARK-15204][SQL] improve nullability inference for Aggregator
## What changes were proposed in this pull request?

TypedAggregateExpression sets nullable based on the schema of the outputEncoder

## How was this patch tested?

Add test in DatasetAggregatorSuite

Author: Koert Kuipers <koert@tresata.com>

Closes #13532 from koertkuipers/feat-aggregator-nullable.
2016-07-04 12:14:14 +08:00
Dongjoon Hyun 88134e7368 [SPARK-16288][SQL] Implement inline table generating function
## What changes were proposed in this pull request?

This PR implements `inline` table generating function.

## How was this patch tested?

Pass the Jenkins tests with new testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13976 from dongjoon-hyun/SPARK-16288.
2016-07-04 01:57:45 +08:00
Dongjoon Hyun 54b27c1797 [SPARK-16278][SPARK-16279][SQL] Implement map_keys/map_values SQL functions
## What changes were proposed in this pull request?

This PR adds `map_keys` and `map_values` SQL functions in order to remove Hive fallback.

## How was this patch tested?

Pass the Jenkins tests including new testcases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13967 from dongjoon-hyun/SPARK-16278.
2016-07-03 16:59:40 +08:00
gatorsmile ea990f9693 [SPARK-16329][SQL] Star Expansion over Table Containing No Column
#### What changes were proposed in this pull request?
Star expansion over a table containing zero column does not work since 1.6. However, it works in Spark 1.5.1. This PR is to fix the issue in the master branch.

For example,
```scala
val rddNoCols = sqlContext.sparkContext.parallelize(1 to 10).map(_ => Row.empty)
val dfNoCols = sqlContext.createDataFrame(rddNoCols, StructType(Seq.empty))
dfNoCols.registerTempTable("temp_table_no_cols")
sqlContext.sql("select * from temp_table_no_cols").show
```

Without the fix, users will get the following the exception:
```
java.lang.IllegalArgumentException: requirement failed
        at scala.Predef$.require(Predef.scala:221)
        at org.apache.spark.sql.catalyst.analysis.UnresolvedStar.expand(unresolved.scala:199)
```

#### How was this patch tested?
Tests are added

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14007 from gatorsmile/starExpansionTableWithZeroColumn.
2016-07-03 16:48:04 +08:00
Dongjoon Hyun 3000b4b29f [MINOR][BUILD] Fix Java linter errors
## What changes were proposed in this pull request?

This PR fixes the minor Java linter errors like the following.
```
-    public int read(char cbuf[], int off, int len) throws IOException {
+    public int read(char[] cbuf, int off, int len) throws IOException {
```

## How was this patch tested?

Manual.
```
$ build/mvn -T 4 -q -DskipTests -Pyarn -Phadoop-2.3 -Pkinesis-asl -Phive -Phive-thriftserver install
$ dev/lint-java
Using `mvn` from path: /usr/local/bin/mvn
Checkstyle checks passed.
```

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14017 from dongjoon-hyun/minor_build_java_linter_error.
2016-07-02 16:31:06 +01:00
Reynold Xin d601894c04 [SPARK-16335][SQL] Structured streaming should fail if source directory does not exist
## What changes were proposed in this pull request?
In structured streaming, Spark does not report errors when the specified directory does not exist. This is a behavior different from the batch mode. This patch changes the behavior to fail if the directory does not exist (when the path is not a glob pattern).

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

Author: Reynold Xin <rxin@databricks.com>

Closes #14002 from rxin/SPARK-16335.
2016-07-01 15:16:04 -07:00
Dongjoon Hyun c55397652a [SPARK-16208][SQL] Add PropagateEmptyRelation optimizer
## What changes were proposed in this pull request?

This PR adds a new logical optimizer, `PropagateEmptyRelation`, to collapse a logical plans consisting of only empty LocalRelations.

**Optimizer Targets**

1. Binary(or Higher)-node Logical Plans
   - Union with all empty children.
   - Join with one or two empty children (including Intersect/Except).
2. Unary-node Logical Plans
   - Project/Filter/Sample/Join/Limit/Repartition with all empty children.
   - Aggregate with all empty children and without AggregateFunction expressions, COUNT.
   - Generate with Explode because other UserDefinedGenerators like Hive UDTF returns results.

**Sample Query**
```sql
WITH t1 AS (SELECT a FROM VALUES 1 t(a)),
     t2 AS (SELECT b FROM VALUES 1 t(b) WHERE 1=2)
SELECT a,b
FROM t1, t2
WHERE a=b
GROUP BY a,b
HAVING a>1
ORDER BY a,b
```

**Before**
```scala
scala> sql("with t1 as (select a from values 1 t(a)), t2 as (select b from values 1 t(b) where 1=2) select a,b from t1, t2 where a=b group by a,b having a>1 order by a,b").explain
== Physical Plan ==
*Sort [a#0 ASC, b#1 ASC], true, 0
+- Exchange rangepartitioning(a#0 ASC, b#1 ASC, 200)
   +- *HashAggregate(keys=[a#0, b#1], functions=[])
      +- Exchange hashpartitioning(a#0, b#1, 200)
         +- *HashAggregate(keys=[a#0, b#1], functions=[])
            +- *BroadcastHashJoin [a#0], [b#1], Inner, BuildRight
               :- *Filter (isnotnull(a#0) && (a#0 > 1))
               :  +- LocalTableScan [a#0]
               +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)))
                  +- *Filter (isnotnull(b#1) && (b#1 > 1))
                     +- LocalTableScan <empty>, [b#1]
```

**After**
```scala
scala> sql("with t1 as (select a from values 1 t(a)), t2 as (select b from values 1 t(b) where 1=2) select a,b from t1, t2 where a=b group by a,b having a>1 order by a,b").explain
== Physical Plan ==
LocalTableScan <empty>, [a#0, b#1]
```

## How was this patch tested?

Pass the Jenkins tests (including a new testsuite).

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13906 from dongjoon-hyun/SPARK-16208.
2016-07-01 22:13:56 +08:00
gatorsmile 0ad6ce7e54 [SPARK-16222][SQL] JDBC Sources - Handling illegal input values for fetchsize and batchsize
#### What changes were proposed in this pull request?
For JDBC data sources, users can specify `batchsize` for multi-row inserts and `fetchsize` for multi-row fetch. A few issues exist:

- The property keys are case sensitive. Thus, the existing test cases for `fetchsize` use incorrect names, `fetchSize`. Basically, the test cases are broken.
- No test case exists for `batchsize`.
- We do not detect the illegal input values for `fetchsize` and `batchsize`.

For example, when `batchsize` is zero, we got the following exception:
```
Job aborted due to stage failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in stage 0.0 (TID 0, localhost): java.lang.ArithmeticException: / by zero
```
when `fetchsize` is less than zero, we got the exception from the underlying JDBC driver:
```
Job aborted due to stage failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in stage 0.0 (TID 0, localhost): org.h2.jdbc.JdbcSQLException: Invalid value "-1" for parameter "rows" [90008-183]
```

This PR fixes all the above issues, and issue the appropriate exceptions when detecting the illegal inputs for `fetchsize` and `batchsize`. Also update the function descriptions.

#### How was this patch tested?
Test cases are fixed and added.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13919 from gatorsmile/jdbcProperties.
2016-07-01 09:54:02 +01:00
Hiroshi Inoue 14cf61e909 [SPARK-16331][SQL] Reduce code generation time
## What changes were proposed in this pull request?
During the code generation, a `LocalRelation` often has a huge `Vector` object as `data`. In the simple example below, a `LocalRelation` has a Vector with 1000000 elements of `UnsafeRow`.

```
val numRows = 1000000
val ds = (1 to numRows).toDS().persist()
benchmark.addCase("filter+reduce") { iter =>
  ds.filter(a => (a & 1) == 0).reduce(_ + _)
}
```

At `TreeNode.transformChildren`, all elements of the vector is unnecessarily iterated to check whether any children exist in the vector since `Vector` is Traversable. This part significantly increases code generation time.

This patch avoids this overhead by checking the number of children before iterating all elements; `LocalRelation` does not have children since it extends `LeafNode`.

The performance of the above example
```
without this patch
Java HotSpot(TM) 64-Bit Server VM 1.8.0_91-b14 on Mac OS X 10.11.5
Intel(R) Core(TM) i5-5257U CPU  2.70GHz
compilationTime:                         Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
filter+reduce                                 4426 / 4533          0.2        4426.0       1.0X

with this patch
compilationTime:                         Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
filter+reduce                                 3117 / 3391          0.3        3116.6       1.0X
```

## How was this patch tested?

using existing unit tests

Author: Hiroshi Inoue <inouehrs@jp.ibm.com>

Closes #14000 from inouehrs/compilation-time-reduction.
2016-06-30 21:47:44 -07:00
Reynold Xin 38f4d6f44e [SPARK-15954][SQL] Disable loading test tables in Python tests
## What changes were proposed in this pull request?
This patch introduces a flag to disable loading test tables in TestHiveSparkSession and disables that in Python. This fixes an issue in which python/run-tests would fail due to failure to load test tables.

Note that these test tables are not used outside of HiveCompatibilitySuite. In the long run we should probably decouple the loading of test tables from the test Hive setup.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #14005 from rxin/SPARK-15954.
2016-06-30 19:02:35 -07:00
petermaxlee 85f2303eca [SPARK-16276][SQL] Implement elt SQL function
## What changes were proposed in this pull request?
This patch implements the elt function, as it is implemented in Hive.

## How was this patch tested?
Added expression unit test in StringExpressionsSuite and end-to-end test in StringFunctionsSuite.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #13966 from petermaxlee/SPARK-16276.
2016-07-01 07:57:48 +08:00
Reynold Xin 3d75a5b2a7 [SPARK-16313][SQL] Spark should not silently drop exceptions in file listing
## What changes were proposed in this pull request?
Spark silently drops exceptions during file listing. This is a very bad behavior because it can mask legitimate errors and the resulting plan will silently have 0 rows. This patch changes it to not silently drop the errors.

## How was this patch tested?
Manually verified.

Author: Reynold Xin <rxin@databricks.com>

Closes #13987 from rxin/SPARK-16313.
2016-06-30 16:51:11 -07:00
petermaxlee fb41670c92 [SPARK-16336][SQL] Suggest doing table refresh upon FileNotFoundException
## What changes were proposed in this pull request?
This patch appends a message to suggest users running refresh table or reloading data frames when Spark sees a FileNotFoundException due to stale, cached metadata.

## How was this patch tested?
Added a unit test for this in MetadataCacheSuite.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14003 from petermaxlee/SPARK-16336.
2016-06-30 16:49:59 -07:00
Dongjoon Hyun 46395db80e [SPARK-16289][SQL] Implement posexplode table generating function
## What changes were proposed in this pull request?

This PR implements `posexplode` table generating function. Currently, master branch raises the following exception for `map` argument. It's different from Hive.

**Before**
```scala
scala> sql("select posexplode(map('a', 1, 'b', 2))").show
org.apache.spark.sql.AnalysisException: No handler for Hive UDF ... posexplode() takes an array as a parameter; line 1 pos 7
```

**After**
```scala
scala> sql("select posexplode(map('a', 1, 'b', 2))").show
+---+---+-----+
|pos|key|value|
+---+---+-----+
|  0|  a|    1|
|  1|  b|    2|
+---+---+-----+
```

For `array` argument, `after` is the same with `before`.
```
scala> sql("select posexplode(array(1, 2, 3))").show
+---+---+
|pos|col|
+---+---+
|  0|  1|
|  1|  2|
|  2|  3|
+---+---+
```

## How was this patch tested?

Pass the Jenkins tests with newly added testcases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13971 from dongjoon-hyun/SPARK-16289.
2016-06-30 12:03:54 -07:00
Sital Kedia 07f46afc73 [SPARK-13850] Force the sorter to Spill when number of elements in th…
## What changes were proposed in this pull request?

Force the sorter to Spill when number of elements in the pointer array reach a certain size. This is to workaround the issue of timSort failing on large buffer size.

## How was this patch tested?

Tested by running a job which was failing without this change due to TimSort bug.

Author: Sital Kedia <skedia@fb.com>

Closes #13107 from sitalkedia/fix_TimSort.
2016-06-30 10:53:18 -07:00
WeichenXu 5344bade8e [SPARK-15820][PYSPARK][SQL] Add Catalog.refreshTable into python API
## What changes were proposed in this pull request?

Add Catalog.refreshTable API into python interface for Spark-SQL.

## How was this patch tested?

Existing test.

Author: WeichenXu <WeichenXu123@outlook.com>

Closes #13558 from WeichenXu123/update_python_sql_interface_refreshTable.
2016-06-30 23:00:39 +08:00
Sean Zhong 5320adc863 [SPARK-16071][SQL] Checks size limit when doubling the array size in BufferHolder
## What changes were proposed in this pull request?

This PR Checks the size limit when doubling the array size in BufferHolder to avoid integer overflow.

## How was this patch tested?

Manual test.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #13829 from clockfly/SPARK-16071_2.
2016-06-30 21:56:34 +08:00
petermaxlee d3af6731fa [SPARK-16274][SQL] Implement xpath_boolean
## What changes were proposed in this pull request?
This patch implements xpath_boolean expression for Spark SQL, a xpath function that returns true or false. The implementation is modelled after Hive's xpath_boolean, except that how the expression handles null inputs. Hive throws a NullPointerException at runtime if either of the input is null. This implementation returns null if either of the input is null.

## How was this patch tested?
Created two new test suites. One for unit tests covering the expression, and the other for end-to-end test in SQL.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #13964 from petermaxlee/SPARK-16274.
2016-06-30 09:27:48 +08:00
Dongjoon Hyun 831a04f5d1 [SPARK-16267][TEST] Replace deprecated CREATE TEMPORARY TABLE ... USING from testsuites.
## What changes were proposed in this pull request?

After SPARK-15674, `DDLStrategy` prints out the following deprecation messages in the testsuites.

```
12:10:53.284 WARN org.apache.spark.sql.execution.SparkStrategies$DDLStrategy:
CREATE TEMPORARY TABLE normal_orc_source USING... is deprecated,
please use CREATE TEMPORARY VIEW viewName USING... instead
```

Total : 40
- JDBCWriteSuite: 14
- DDLSuite: 6
- TableScanSuite: 6
- ParquetSourceSuite: 5
- OrcSourceSuite: 2
- SQLQuerySuite: 2
- HiveCommandSuite: 2
- JsonSuite: 1
- PrunedScanSuite: 1
- FilteredScanSuite  1

This PR replaces `CREATE TEMPORARY TABLE` with `CREATE TEMPORARY VIEW` in order to remove the deprecation messages in the above testsuites except `DDLSuite`, `SQLQuerySuite`, `HiveCommandSuite`.

The Jenkins results shows only remaining 10 messages.

https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/61422/consoleFull

## How was this patch tested?

This is a testsuite-only change.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13956 from dongjoon-hyun/SPARK-16267.
2016-06-29 17:29:17 -07:00
Wenchen Fan d063898beb [SPARK-16134][SQL] optimizer rules for typed filter
## What changes were proposed in this pull request?

This PR adds 3 optimizer rules for typed filter:

1. push typed filter down through `SerializeFromObject` and eliminate the deserialization in filter condition.
2. pull typed filter up through `SerializeFromObject` and eliminate the deserialization in filter condition.
3. combine adjacent typed filters and share the deserialized object among all the condition expressions.

This PR also adds `TypedFilter` logical plan, to separate it from normal filter, so that the concept is more clear and it's easier to write optimizer rules.

## How was this patch tested?

`TypedFilterOptimizationSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #13846 from cloud-fan/filter.
2016-06-30 08:15:08 +08:00
Dongjoon Hyun 2eaabfa414 [SPARK-16228][SQL] HiveSessionCatalog should return double-param functions for decimal param lookups
## What changes were proposed in this pull request?

This PR supports a fallback lookup by casting `DecimalType` into `DoubleType` for the external functions with `double`-type parameter.

**Reported Error Scenarios**
```scala
scala> sql("select percentile(value, 0.5) from values 1,2,3 T(value)")
org.apache.spark.sql.AnalysisException: ... No matching method for class org.apache.hadoop.hive.ql.udf.UDAFPercentile with (int, decimal(38,18)). Possible choices: _FUNC_(bigint, array<double>)  _FUNC_(bigint, double)  ; line 1 pos 7

scala> sql("select percentile_approx(value, 0.5) from values 1.0,2.0,3.0 T(value)")
org.apache.spark.sql.AnalysisException: ... Only a float/double or float/double array argument is accepted as parameter 2, but decimal(38,18) was passed instead.; line 1 pos 7
```

## How was this patch tested?

Pass the Jenkins tests (including a new testcase).

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13930 from dongjoon-hyun/SPARK-16228.
2016-06-29 16:08:10 -07:00
Eric Liang 23c58653f9 [SPARK-16238] Metrics for generated method and class bytecode size
## What changes were proposed in this pull request?

This extends SPARK-15860 to include metrics for the actual bytecode size of janino-generated methods. They can be accessed in the same way as any other codahale metric, e.g.

```
scala> org.apache.spark.metrics.source.CodegenMetrics.METRIC_GENERATED_CLASS_BYTECODE_SIZE.getSnapshot().getValues()
res7: Array[Long] = Array(532, 532, 532, 542, 1479, 2670, 3585, 3585)

scala> org.apache.spark.metrics.source.CodegenMetrics.METRIC_GENERATED_METHOD_BYTECODE_SIZE.getSnapshot().getValues()
res8: Array[Long] = Array(5, 5, 5, 5, 10, 10, 10, 10, 15, 15, 15, 38, 63, 79, 88, 94, 94, 94, 132, 132, 165, 165, 220, 220)
```

## How was this patch tested?

Small unit test, also verified manually that the performance impact is minimal (<10%). hvanhovell

Author: Eric Liang <ekl@databricks.com>

Closes #13934 from ericl/spark-16238.
2016-06-29 15:07:32 -07:00
Dongjoon Hyun 9b1b3ae771 [SPARK-16006][SQL] Attemping to write empty DataFrame with no fields throws non-intuitive exception
## What changes were proposed in this pull request?

This PR allows `emptyDataFrame.write` since the user didn't specify any partition columns.

**Before**
```scala
scala> spark.emptyDataFrame.write.parquet("/tmp/t1")
org.apache.spark.sql.AnalysisException: Cannot use all columns for partition columns;
scala> spark.emptyDataFrame.write.csv("/tmp/t1")
org.apache.spark.sql.AnalysisException: Cannot use all columns for partition columns;
```

After this PR, there occurs no exceptions and the created directory has only one file, `_SUCCESS`, as expected.

## How was this patch tested?

Pass the Jenkins tests including updated test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13730 from dongjoon-hyun/SPARK-16006.
2016-06-29 15:00:41 -07:00
Yin Huai 8b5a8b25b9 [SPARK-16301] [SQL] The analyzer rule for resolving using joins should respect the case sensitivity setting.
## What changes were proposed in this pull request?
The analyzer rule for resolving using joins should respect the case sensitivity setting.

## How was this patch tested?
New tests in ResolveNaturalJoinSuite

Author: Yin Huai <yhuai@databricks.com>

Closes #13977 from yhuai/SPARK-16301.
2016-06-29 14:42:58 -07:00
hyukjinkwon cb1b9d34f3 [SPARK-14480][SQL] Remove meaningless StringIteratorReader for CSV data source.
## What changes were proposed in this pull request?

This PR removes meaningless `StringIteratorReader` for CSV data source.

In `CSVParser.scala`, there is an `Reader` wrapping `Iterator` but there are two problems by this.

Firstly, it was actually not faster than processing line by line with Iterator due to additional logics to wrap `Iterator` to `Reader`.
Secondly, this brought a bit of complexity because it needs additional logics to allow every line to be read bytes by bytes. So, it was pretty difficult to figure out issues about parsing, (eg. SPARK-14103).

A benchmark was performed manually and the results were below:

- Original codes with Reader wrapping Iterator

|End-to-end (ns)  |   Parse Time (ns) |
|-----------------------|------------------------|
|14116265034      |2008277960        |

- New codes with Iterator

|End-to-end (ns)  |   Parse Time (ns) |
|-----------------------|------------------------|
|13451699644      | 1549050564       |

For the details for the environment, dataset and methods, please refer the JIRA ticket.

## How was this patch tested?

Existing tests should cover this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #13808 from HyukjinKwon/SPARK-14480-small.
2016-06-29 11:42:51 -07:00
gatorsmile 7ee9e39cb4 [SPARK-16157][SQL] Add New Methods for comments in StructField and StructType
#### What changes were proposed in this pull request?
Based on the previous discussion with cloud-fan hvanhovell in another related PR https://github.com/apache/spark/pull/13764#discussion_r67994276, it looks reasonable to add convenience methods for users to add `comment` when defining `StructField`.

Currently, the column-related `comment` attribute is stored in `Metadata` of `StructField`. For example, users can add the `comment` attribute using the following way:
```Scala
StructType(
  StructField(
    "cl1",
    IntegerType,
    nullable = false,
    new MetadataBuilder().putString("comment", "test").build()) :: Nil)
```
This PR is to add more user friendly methods for the `comment` attribute when defining a `StructField`. After the changes, users are provided three different ways to do it:
```Scala
val struct = (new StructType)
  .add("a", "int", true, "test1")

val struct = (new StructType)
  .add("c", StringType, true, "test3")

val struct = (new StructType)
  .add(StructField("d", StringType).withComment("test4"))
```

#### How was this patch tested?
Added test cases:
- `DataTypeSuite` is for testing three types of API changes,
- `DataFrameReaderWriterSuite` is for parquet, json and csv formats - using in-memory catalog
- `OrcQuerySuite.scala` is for orc format using Hive-metastore

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13860 from gatorsmile/newMethodForComment.
2016-06-29 19:36:21 +08:00
Cheng Lian d1e8108854 [SPARK-16291][SQL] CheckAnalysis should capture nested aggregate functions that reference no input attributes
## What changes were proposed in this pull request?

`MAX(COUNT(*))` is invalid since aggregate expression can't be nested within another aggregate expression. This case should be captured at analysis phase, but somehow sneaks off to runtime.

The reason is that when checking aggregate expressions in `CheckAnalysis`, a checking branch treats all expressions that reference no input attributes as valid ones. However, `MAX(COUNT(*))` is translated into `MAX(COUNT(1))` at analysis phase and also references no input attribute.

This PR fixes this issue by removing the aforementioned branch.

## How was this patch tested?

New test case added in `AnalysisErrorSuite`.

Author: Cheng Lian <lian@databricks.com>

Closes #13968 from liancheng/spark-16291-nested-agg-functions.
2016-06-29 19:08:36 +08:00
Holden Karau 757dc2c09d [TRIVIAL][DOCS][STREAMING][SQL] The return type mentioned in the Javadoc is incorrect for toJavaRDD, …
## What changes were proposed in this pull request?

Change the return type mentioned in the JavaDoc for `toJavaRDD` / `javaRDD` to match the actual return type & be consistent with the scala rdd return type.

## How was this patch tested?

Docs only change.

Author: Holden Karau <holden@us.ibm.com>

Closes #13954 from holdenk/trivial-streaming-tojavardd-doc-fix.
2016-06-29 01:52:20 -07:00
petermaxlee 153c2f9ac1 [SPARK-16271][SQL] Implement Hive's UDFXPathUtil
## What changes were proposed in this pull request?
This patch ports Hive's UDFXPathUtil over to Spark, which can be used to implement xpath functionality in Spark in the near future.

## How was this patch tested?
Added two new test suites UDFXPathUtilSuite and ReusableStringReaderSuite. They have been ported over from Hive (but rewritten in Scala in order to leverage ScalaTest).

Author: petermaxlee <petermaxlee@gmail.com>

Closes #13961 from petermaxlee/xpath.
2016-06-28 21:07:52 -07:00
Reynold Xin 363bcedeea [SPARK-16248][SQL] Whitelist the list of Hive fallback functions
## What changes were proposed in this pull request?
This patch removes the blind fallback into Hive for functions. Instead, it creates a whitelist and adds only a small number of functions to the whitelist, i.e. the ones we intend to support in the long run in Spark.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #13939 from rxin/hive-whitelist.
2016-06-28 19:36:53 -07:00
Burak Yavuz 5545b79109 [MINOR][DOCS][STRUCTURED STREAMING] Minor doc fixes around DataFrameWriter and DataStreamWriter
## What changes were proposed in this pull request?

Fixes a couple old references to `DataFrameWriter.startStream` to `DataStreamWriter.start

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #13952 from brkyvz/minor-doc-fix.
2016-06-28 17:02:16 -07:00
Wenchen Fan 8a977b0654 [SPARK-16100][SQL] fix bug when use Map as the buffer type of Aggregator
## What changes were proposed in this pull request?

The root cause is in `MapObjects`. Its parameter `loopVar` is not declared as child, but sometimes can be same with `lambdaFunction`(e.g. the function that takes `loopVar` and produces `lambdaFunction` may be `identity`), which is a child. This brings trouble when call `withNewChildren`, it may mistakenly treat `loopVar` as a child and cause `IndexOutOfBoundsException: 0` later.

This PR fixes this bug by simply pulling out the paremters from `LambdaVariable` and pass them to `MapObjects` directly.

## How was this patch tested?

new test in `DatasetAggregatorSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #13835 from cloud-fan/map-objects.
2016-06-29 06:39:28 +08:00
gatorsmile 25520e9762 [SPARK-16236][SQL] Add Path Option back to Load API in DataFrameReader
#### What changes were proposed in this pull request?
koertkuipers identified the PR https://github.com/apache/spark/pull/13727/ changed the behavior of `load` API. After the change, the `load` API does not add the value of `path` into the `options`. Thank you!

This PR is to add the option `path` back to `load()` API in `DataFrameReader`, if and only if users specify one and only one `path` in the `load` API. For example, users can see the `path` option after the following API call,
```Scala
spark.read
  .format("parquet")
  .load("/test")
```

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13933 from gatorsmile/optionPath.
2016-06-28 15:32:45 -07:00
Wenchen Fan 1f2776df6e [SPARK-16181][SQL] outer join with isNull filter may return wrong result
## What changes were proposed in this pull request?

The root cause is: the output attributes of outer join are derived from its children, while they are actually different attributes(outer join can return null).

We have already added some special logic to handle it, e.g. `PushPredicateThroughJoin` won't push down predicates through outer join side, `FixNullability`.

This PR adds one more special logic in `FoldablePropagation`.

## How was this patch tested?

new test in `DataFrameSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #13884 from cloud-fan/bug.
2016-06-28 10:26:01 -07:00
Prashant Sharma f6b497fcdd [SPARK-16128][SQL] Allow setting length of characters to be truncated to, in Dataset.show function.
## What changes were proposed in this pull request?

Allowing truncate to a specific number of character is convenient at times, especially while operating from the REPL. Sometimes those last few characters make all the difference, and showing everything brings in whole lot of noise.

## How was this patch tested?
Existing tests. + 1 new test in DataFrameSuite.

For SparkR and pyspark, existing tests and manual testing.

Author: Prashant Sharma <prashsh1@in.ibm.com>
Author: Prashant Sharma <prashant@apache.org>

Closes #13839 from ScrapCodes/add_truncateTo_DF.show.
2016-06-28 17:11:06 +05:30
gatorsmile 4cbf611c1d [SPARK-16202][SQL][DOC] Correct The Description of CreatableRelationProvider's createRelation
#### What changes were proposed in this pull request?
The API description of `createRelation` in `CreatableRelationProvider` is misleading. The current description only expects users to return the relation.

```Scala
trait CreatableRelationProvider {
  def createRelation(
      sqlContext: SQLContext,
      mode: SaveMode,
      parameters: Map[String, String],
      data: DataFrame): BaseRelation
}
```

However, the major goal of this API should also include saving the `DataFrame`.

Since this API is critical for Data Source API developers, this PR is to correct the description.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13903 from gatorsmile/readUnderscoreFiles.
2016-06-27 23:12:17 -07:00
Dongjoon Hyun a0da854fb3 [SPARK-16221][SQL] Redirect Parquet JUL logger via SLF4J for WRITE operations
## What changes were proposed in this pull request?

[SPARK-8118](https://github.com/apache/spark/pull/8196) implements redirecting Parquet JUL logger via SLF4J, but it is currently applied only when READ operations occurs. If users use only WRITE operations, there occurs many Parquet logs.

This PR makes the redirection work on WRITE operations, too.

**Before**
```scala
scala> spark.range(10).write.format("parquet").mode("overwrite").save("/tmp/p")
SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
SLF4J: Defaulting to no-operation (NOP) logger implementation
SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further details.
Jun 26, 2016 9:04:38 PM INFO: org.apache.parquet.hadoop.codec.CodecConfig: Compression: SNAPPY
............ about 70 lines Parquet Log .............
scala> spark.range(10).write.format("parquet").mode("overwrite").save("/tmp/p")
............ about 70 lines Parquet Log .............
```

**After**
```scala
scala> spark.range(10).write.format("parquet").mode("overwrite").save("/tmp/p")
SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
SLF4J: Defaulting to no-operation (NOP) logger implementation
SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further details.
scala> spark.range(10).write.format("parquet").mode("overwrite").save("/tmp/p")
```

This PR also fixes some typos.

## How was this patch tested?

Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13918 from dongjoon-hyun/SPARK-16221.
2016-06-28 13:01:18 +08:00
Herman van Hovell 02a029df43 [SPARK-16220][SQL] Add scope to show functions
## What changes were proposed in this pull request?
Spark currently shows all functions when issue a `SHOW FUNCTIONS` command. This PR refines the `SHOW FUNCTIONS` command by allowing users to select all functions, user defined function or system functions. The following syntax can be used:

**ALL** (default)
```SHOW FUNCTIONS```
```SHOW ALL FUNCTIONS```

**SYSTEM**
```SHOW SYSTEM FUNCTIONS```

**USER**
```SHOW USER FUNCTIONS```
## How was this patch tested?
Updated tests and added tests to the DDLSuite

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

Closes #13929 from hvanhovell/SPARK-16220.
2016-06-27 16:57:34 -07:00
Bill Chambers c48c8ebc0a [SPARK-16220][SQL] Revert Change to Bring Back SHOW FUNCTIONS Functionality
## What changes were proposed in this pull request?

- Fix tests regarding show functions functionality
- Revert `catalog.ListFunctions` and `SHOW FUNCTIONS` to return to `Spark 1.X` functionality.

Cherry picked changes from this PR: https://github.com/apache/spark/pull/13413/files

## How was this patch tested?

Unit tests.

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

Closes #13916 from anabranch/master.
2016-06-27 11:50:34 -07:00
Takeshi YAMAMURO 3e4e868c85 [SPARK-16135][SQL] Remove hashCode and euqals in ArrayBasedMapData
## What changes were proposed in this pull request?
This pr is to remove `hashCode` and `equals` in `ArrayBasedMapData` because the type cannot be used as join keys, grouping keys, or in equality tests.

## How was this patch tested?
Add a new test suite `MapDataSuite` for comparison tests.

Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes #13847 from maropu/UnsafeMapTest.
2016-06-27 21:45:22 +08:00
Dongjoon Hyun 11f420b4bb [SPARK-10591][SQL][TEST] Add a testcase to ensure if checkAnswer handles map correctly
## What changes were proposed in this pull request?

This PR adds a testcase to ensure if `checkAnswer` handles Map type correctly.

## How was this patch tested?

Pass the jenkins tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13913 from dongjoon-hyun/SPARK-10591.
2016-06-27 19:04:50 +08:00
Felix Cheung 30b182bcc0 [SPARK-16184][SPARKR] conf API for SparkSession
## What changes were proposed in this pull request?

Add `conf` method to get Runtime Config from SparkSession

## How was this patch tested?

unit tests, manual tests

This is how it works in sparkR shell:
```
 SparkSession available as 'spark'.
> conf()
$hive.metastore.warehouse.dir
[1] "file:/opt/spark-2.0.0-bin-hadoop2.6/R/spark-warehouse"

$spark.app.id
[1] "local-1466749575523"

$spark.app.name
[1] "SparkR"

$spark.driver.host
[1] "10.0.2.1"

$spark.driver.port
[1] "45629"

$spark.executorEnv.LD_LIBRARY_PATH
[1] "$LD_LIBRARY_PATH:/usr/lib/R/lib:/usr/lib/x86_64-linux-gnu:/usr/lib/jvm/default-java/jre/lib/amd64/server"

$spark.executor.id
[1] "driver"

$spark.home
[1] "/opt/spark-2.0.0-bin-hadoop2.6"

$spark.master
[1] "local[*]"

$spark.sql.catalogImplementation
[1] "hive"

$spark.submit.deployMode
[1] "client"

> conf("spark.master")
$spark.master
[1] "local[*]"

```

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #13885 from felixcheung/rconf.
2016-06-26 13:10:43 -07:00
Sital Kedia bf665a9586 [SPARK-15958] Make initial buffer size for the Sorter configurable
## What changes were proposed in this pull request?

Currently the initial buffer size in the sorter is hard coded inside the code and is too small for large workload. As a result, the sorter spends significant time expanding the buffer size and copying the data. It would be useful to have it configurable.

## How was this patch tested?

Tested by running a job on the cluster.

Author: Sital Kedia <skedia@fb.com>

Closes #13699 from sitalkedia/config_sort_buffer_upstream.
2016-06-25 09:13:39 +01:00
Dongjoon Hyun a7d29499dc [SPARK-16186] [SQL] Support partition batch pruning with IN predicate in InMemoryTableScanExec
## What changes were proposed in this pull request?

One of the most frequent usage patterns for Spark SQL is using **cached tables**. This PR improves `InMemoryTableScanExec` to handle `IN` predicate efficiently by pruning partition batches. Of course, the performance improvement varies over the queries and the datasets. But, for the following simple query, the query duration in Spark UI goes from 9 seconds to 50~90ms. It's about over 100 times faster.

**Before**
```scala
$ bin/spark-shell --driver-memory 6G
scala> val df = spark.range(2000000000)
scala> df.createOrReplaceTempView("t")
scala> spark.catalog.cacheTable("t")
scala> sql("select id from t where id = 1").collect()    // About 2 mins
scala> sql("select id from t where id = 1").collect()    // less than 90ms
scala> sql("select id from t where id in (1,2,3)").collect()  // 9 seconds
```

**After**
```scala
scala> sql("select id from t where id in (1,2,3)").collect() // less than 90ms
```

This PR has impacts over 35 queries of TPC-DS if the tables are cached.
Note that this optimization is applied for `IN`.  To apply `IN` predicate having more than 10 items, `spark.sql.optimizer.inSetConversionThreshold` option should be increased.

## How was this patch tested?

Pass the Jenkins tests (including new testcases).

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13887 from dongjoon-hyun/SPARK-16186.
2016-06-24 22:34:31 -07:00
Takeshi YAMAMURO d2e44d7db8 [SPARK-16192][SQL] Add type checks in CollectSet
## What changes were proposed in this pull request?
`CollectSet` cannot have map-typed data because MapTypeData does not implement `equals`.
So, this pr is to add type checks in `CheckAnalysis`.

## How was this patch tested?
Added tests to check failures when we found map-typed data in `CollectSet`.

Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes #13892 from maropu/SPARK-16192.
2016-06-24 21:07:03 -07:00
Dilip Biswal 9053054c7f [SPARK-16195][SQL] Allow users to specify empty over clause in window expressions through dataset API
## What changes were proposed in this pull request?
Allow to specify empty over clause in window expressions through dataset API

In SQL, its allowed to specify an empty OVER clause in the window expression.

```SQL
select area, sum(product) over () as c from windowData
where product > 3 group by area, product
having avg(month) > 0 order by avg(month), product
```
In this case the analytic function sum is presented based on all the rows of the result set

Currently its not allowed through dataset API and is handled in this PR.

## How was this patch tested?

Added a new test in DataframeWindowSuite

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

Closes #13897 from dilipbiswal/spark-empty-over.
2016-06-24 17:27:33 -07:00
Dongjoon Hyun e5d0928e24 [SPARK-16173] [SQL] Can't join describe() of DataFrame in Scala 2.10
## What changes were proposed in this pull request?

This PR fixes `DataFrame.describe()` by forcing materialization to make the `Seq` serializable. Currently, `describe()` of DataFrame throws `Task not serializable` Spark exceptions when joining in Scala 2.10.

## How was this patch tested?

Manual. (After building with Scala 2.10, test on `bin/spark-shell` and `bin/pyspark`.)

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13900 from dongjoon-hyun/SPARK-16173.
2016-06-24 17:26:39 -07:00
Davies Liu 20768dade2 Revert "[SPARK-16186] [SQL] Support partition batch pruning with IN predicate in InMemoryTableScanExec"
This reverts commit a65bcbc27d.
2016-06-24 17:21:18 -07:00
Dongjoon Hyun a65bcbc27d [SPARK-16186] [SQL] Support partition batch pruning with IN predicate in InMemoryTableScanExec
## What changes were proposed in this pull request?

One of the most frequent usage patterns for Spark SQL is using **cached tables**. This PR improves `InMemoryTableScanExec` to handle `IN` predicate efficiently by pruning partition batches. Of course, the performance improvement varies over the queries and the datasets. But, for the following simple query, the query duration in Spark UI goes from 9 seconds to 50~90ms. It's about over 100 times faster.

**Before**
```scala
$ bin/spark-shell --driver-memory 6G
scala> val df = spark.range(2000000000)
scala> df.createOrReplaceTempView("t")
scala> spark.catalog.cacheTable("t")
scala> sql("select id from t where id = 1").collect()    // About 2 mins
scala> sql("select id from t where id = 1").collect()    // less than 90ms
scala> sql("select id from t where id in (1,2,3)").collect()  // 9 seconds
```

**After**
```scala
scala> sql("select id from t where id in (1,2,3)").collect() // less than 90ms
```

This PR has impacts over 35 queries of TPC-DS if the tables are cached.
Note that this optimization is applied for `IN`.  To apply `IN` predicate having more than 10 items, `spark.sql.optimizer.inSetConversionThreshold` option should be increased.

## How was this patch tested?

Pass the Jenkins tests (including new testcases).

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13887 from dongjoon-hyun/SPARK-16186.
2016-06-24 17:13:13 -07:00
Davies Liu 4435de1bd3 [SPARK-16179][PYSPARK] fix bugs for Python udf in generate
## What changes were proposed in this pull request?

This PR fix the bug when Python UDF is used in explode (generator), GenerateExec requires that all the attributes in expressions should be resolvable from children when creating, we should replace the children first, then replace it's expressions.

```
>>> df.select(explode(f(*df))).show()
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/home/vlad/dev/spark/python/pyspark/sql/dataframe.py", line 286, in show
    print(self._jdf.showString(n, truncate))
  File "/home/vlad/dev/spark/python/lib/py4j-0.10.1-src.zip/py4j/java_gateway.py", line 933, in __call__
  File "/home/vlad/dev/spark/python/pyspark/sql/utils.py", line 63, in deco
    return f(*a, **kw)
  File "/home/vlad/dev/spark/python/lib/py4j-0.10.1-src.zip/py4j/protocol.py", line 312, in get_return_value
py4j.protocol.Py4JJavaError: An error occurred while calling o52.showString.
: org.apache.spark.sql.catalyst.errors.package$TreeNodeException: makeCopy, tree:
Generate explode(<lambda>(_1#0L)), false, false, [col#15L]
+- Scan ExistingRDD[_1#0L]

	at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:50)
	at org.apache.spark.sql.catalyst.trees.TreeNode.makeCopy(TreeNode.scala:387)
	at org.apache.spark.sql.execution.SparkPlan.makeCopy(SparkPlan.scala:69)
	at org.apache.spark.sql.execution.SparkPlan.makeCopy(SparkPlan.scala:45)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpressionsDown(QueryPlan.scala:177)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpressions(QueryPlan.scala:144)
	at org.apache.spark.sql.execution.python.ExtractPythonUDFs$.org$apache$spark$sql$execution$python$ExtractPythonUDFs$$extract(ExtractPythonUDFs.scala:153)
	at org.apache.spark.sql.execution.python.ExtractPythonUDFs$$anonfun$apply$2.applyOrElse(ExtractPythonUDFs.scala:114)
	at org.apache.spark.sql.execution.python.ExtractPythonUDFs$$anonfun$apply$2.applyOrElse(ExtractPythonUDFs.scala:113)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:301)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:301)
	at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:69)
	at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:300)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:298)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:298)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:321)
	at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:179)
	at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:319)
	at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:298)
	at org.apache.spark.sql.execution.python.ExtractPythonUDFs$.apply(ExtractPythonUDFs.scala:113)
	at org.apache.spark.sql.execution.python.ExtractPythonUDFs$.apply(ExtractPythonUDFs.scala:93)
	at org.apache.spark.sql.execution.QueryExecution$$anonfun$prepareForExecution$1.apply(QueryExecution.scala:95)
	at org.apache.spark.sql.execution.QueryExecution$$anonfun$prepareForExecution$1.apply(QueryExecution.scala:95)
	at scala.collection.LinearSeqOptimized$class.foldLeft(LinearSeqOptimized.scala:124)
	at scala.collection.immutable.List.foldLeft(List.scala:84)
	at org.apache.spark.sql.execution.QueryExecution.prepareForExecution(QueryExecution.scala:95)
	at org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:85)
	at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:85)
	at org.apache.spark.sql.Dataset.withTypedCallback(Dataset.scala:2557)
	at org.apache.spark.sql.Dataset.head(Dataset.scala:1923)
	at org.apache.spark.sql.Dataset.take(Dataset.scala:2138)
	at org.apache.spark.sql.Dataset.showString(Dataset.scala:239)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:237)
	at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
	at py4j.Gateway.invoke(Gateway.java:280)
	at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:128)
	at py4j.commands.CallCommand.execute(CallCommand.java:79)
	at py4j.GatewayConnection.run(GatewayConnection.java:211)
	at java.lang.Thread.run(Thread.java:745)
Caused by: java.lang.reflect.InvocationTargetException
	at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
	at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
	at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
	at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$makeCopy$1$$anonfun$apply$13.apply(TreeNode.scala:413)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$makeCopy$1$$anonfun$apply$13.apply(TreeNode.scala:413)
	at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:69)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$makeCopy$1.apply(TreeNode.scala:412)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$makeCopy$1.apply(TreeNode.scala:387)
	at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:49)
	... 42 more
Caused by: org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: pythonUDF0#20
	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)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:279)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:279)
	at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:69)
	at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:278)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:284)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:284)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:321)
	at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:179)
	at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:319)
	at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:284)
	at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:268)
	at org.apache.spark.sql.catalyst.expressions.BindReferences$.bindReference(BoundAttribute.scala:87)
	at org.apache.spark.sql.execution.GenerateExec.<init>(GenerateExec.scala:63)
	... 52 more
Caused by: java.lang.RuntimeException: Couldn't find pythonUDF0#20 in [_1#0L]
	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)
	... 67 more
```

## How was this patch tested?

Added regression tests.

Author: Davies Liu <davies@databricks.com>

Closes #13883 from davies/udf_in_generate.
2016-06-24 15:20:39 -07:00
Reynold Xin 5f8de21606 [SQL][MINOR] Simplify data source predicate filter translation.
## What changes were proposed in this pull request?
This is a small patch to rewrite the predicate filter translation in DataSourceStrategy. The original code used excessive functional constructs (e.g. unzip) and was very difficult to understand.

## How was this patch tested?
Should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #13889 from rxin/simplify-predicate-filter.
2016-06-24 14:44:24 -07:00
Sean Owen 158af162ea [SPARK-16129][CORE][SQL] Eliminate direct use of commons-lang classes in favor of commons-lang3
## What changes were proposed in this pull request?

Replace use of `commons-lang` in favor of `commons-lang3` and forbid the former via scalastyle; remove `NotImplementedException` from `comons-lang` in favor of JDK `UnsupportedOperationException`

## How was this patch tested?

Jenkins tests

Author: Sean Owen <sowen@cloudera.com>

Closes #13843 from srowen/SPARK-16129.
2016-06-24 10:35:54 +01:00
Cheng Lian 2d2f607bfa [SPARK-13709][SQL] Initialize deserializer with both table and partition properties when reading partitioned tables
## What changes were proposed in this pull request?

When reading partitions of a partitioned Hive SerDe table, we only initializes the deserializer using partition properties. However, for SerDes like `AvroSerDe`, essential properties (e.g. Avro schema information) may be defined in table properties. We should merge both table properties and partition properties before initializing the deserializer.

Note that an individual partition may have different properties than the one defined in the table properties (e.g. partitions within a table can have different SerDes). Thus, for any property key defined in both partition and table properties, the value set in partition properties wins.

## How was this patch tested?

New test case added in `QueryPartitionSuite`.

Author: Cheng Lian <lian@databricks.com>

Closes #13865 from liancheng/spark-13709-partitioned-avro-table.
2016-06-23 23:11:46 -07:00
Wenchen Fan 6a3c6276f5 [SQL][MINOR] ParserUtils.operationNotAllowed should throw exception directly
## What changes were proposed in this pull request?

It's weird that `ParserUtils.operationNotAllowed` returns an exception and the caller throw it.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #13874 from cloud-fan/style.
2016-06-23 20:20:55 -07:00
Sameer Agarwal cc71d4fa37 [SPARK-16123] Avoid NegativeArraySizeException while reserving additional capacity in VectorizedColumnReader
## What changes were proposed in this pull request?

This patch fixes an overflow bug in vectorized parquet reader where both off-heap and on-heap variants of `ColumnVector.reserve()` can unfortunately overflow while reserving additional capacity during reads.

## How was this patch tested?

Manual Tests

Author: Sameer Agarwal <sameer@databricks.com>

Closes #13832 from sameeragarwal/negative-array.
2016-06-23 18:21:41 -07:00
Dongjoon Hyun 264bc63623 [SPARK-16165][SQL] Fix the update logic for InMemoryTableScanExec.readBatches
## What changes were proposed in this pull request?

Currently, `readBatches` accumulator of `InMemoryTableScanExec` is updated only when `spark.sql.inMemoryColumnarStorage.partitionPruning` is true. Although this metric is used for only testing purpose, we had better have correct metric without considering SQL options.

## How was this patch tested?

Pass the Jenkins tests (including a new testcase).

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13870 from dongjoon-hyun/SPARK-16165.
2016-06-24 07:19:20 +08:00
Shixiong Zhu 0e4bdebece [SPARK-15443][SQL] Fix 'explain' for streaming Dataset
## What changes were proposed in this pull request?

- Fix the `explain` command for streaming Dataset/DataFrame. E.g.,
```
== Parsed Logical Plan ==
'SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7]
+- 'MapElements <function1>, obj#6: java.lang.String
   +- 'DeserializeToObject unresolveddeserializer(createexternalrow(getcolumnbyordinal(0, StringType).toString, StructField(value,StringType,true))), obj#5: org.apache.spark.sql.Row
      +- Filter <function1>.apply
         +- StreamingRelation FileSource[/Users/zsx/stream], [value#0]

== Analyzed Logical Plan ==
value: string
SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7]
+- MapElements <function1>, obj#6: java.lang.String
   +- DeserializeToObject createexternalrow(value#0.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row
      +- Filter <function1>.apply
         +- StreamingRelation FileSource[/Users/zsx/stream], [value#0]

== Optimized Logical Plan ==
SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7]
+- MapElements <function1>, obj#6: java.lang.String
   +- DeserializeToObject createexternalrow(value#0.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row
      +- Filter <function1>.apply
         +- StreamingRelation FileSource[/Users/zsx/stream], [value#0]

== Physical Plan ==
*SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7]
+- *MapElements <function1>, obj#6: java.lang.String
   +- *DeserializeToObject createexternalrow(value#0.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row
      +- *Filter <function1>.apply
         +- StreamingRelation FileSource[/Users/zsx/stream], [value#0]
```

- Add `StreamingQuery.explain` to display the last execution plan. E.g.,
```
== Parsed Logical Plan ==
SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7]
+- MapElements <function1>, obj#6: java.lang.String
   +- DeserializeToObject createexternalrow(value#12.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row
      +- Filter <function1>.apply
         +- Relation[value#12] text

== Analyzed Logical Plan ==
value: string
SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7]
+- MapElements <function1>, obj#6: java.lang.String
   +- DeserializeToObject createexternalrow(value#12.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row
      +- Filter <function1>.apply
         +- Relation[value#12] text

== Optimized Logical Plan ==
SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7]
+- MapElements <function1>, obj#6: java.lang.String
   +- DeserializeToObject createexternalrow(value#12.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row
      +- Filter <function1>.apply
         +- Relation[value#12] text

== Physical Plan ==
*SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7]
+- *MapElements <function1>, obj#6: java.lang.String
   +- *DeserializeToObject createexternalrow(value#12.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row
      +- *Filter <function1>.apply
         +- *Scan text [value#12] Format: org.apache.spark.sql.execution.datasources.text.TextFileFormat1836ab91, InputPaths: file:/Users/zsx/stream/a.txt, file:/Users/zsx/stream/b.txt, file:/Users/zsx/stream/c.txt, PushedFilters: [], ReadSchema: struct<value:string>
```

## How was this patch tested?

The added unit tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #13815 from zsxwing/sdf-explain.
2016-06-23 16:04:16 -07:00
Dongjoon Hyun 91b1ef28d1 [SPARK-16164][SQL] Update CombineFilters to try to construct predicates with child predicate first
## What changes were proposed in this pull request?

This PR changes `CombineFilters` to compose the final predicate condition by using (`child predicate` AND `parent predicate`) instead of (`parent predicate` AND `child predicate`). This is a best effort approach. Some other optimization rules may destroy this order by reorganizing conjunctive predicates.

**Reported Error Scenario**
Chris McCubbin reported a bug when he used StringIndexer in an ML pipeline with additional filters. It seems that during filter pushdown, we changed the ordering in the logical plan.
```scala
import org.apache.spark.ml.feature._
val df1 = (0 until 3).map(_.toString).toDF
val indexer = new StringIndexer()
  .setInputCol("value")
  .setOutputCol("idx")
  .setHandleInvalid("skip")
  .fit(df1)
val df2 = (0 until 5).map(_.toString).toDF
val predictions = indexer.transform(df2)
predictions.show() // this is okay
predictions.where('idx > 2).show() // this will throw an exception
```

Please see the notebook at https://databricks-prod-cloudfront.cloud.databricks.com/public/4027ec902e239c93eaaa8714f173bcfc/1233855/2159162931615821/588180/latest.html for error messages.

## How was this patch tested?

Pass the Jenkins tests (including a new testcase).

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13872 from dongjoon-hyun/SPARK-16164.
2016-06-23 15:27:43 -07:00
Davies Liu 10396d9505 [SPARK-16163] [SQL] Cache the statistics for logical plans
## What changes were proposed in this pull request?

This calculation of statistics is not trivial anymore, it could be very slow on large query (for example, TPC-DS Q64 took several minutes to plan).

During the planning of a query, the statistics of any logical plan should not change (even InMemoryRelation), so we should use `lazy val` to cache the statistics.

For InMemoryRelation, the statistics could be updated after materialization, it's only useful when used in another query (before planning), because once we finished the planning, the statistics will not be used anymore.

## How was this patch tested?

Testsed with TPC-DS Q64, it could be planned in a second after the patch.

Author: Davies Liu <davies@databricks.com>

Closes #13871 from davies/fix_statistics.
2016-06-23 11:48:48 -07:00
Shixiong Zhu d85bb10ce4 [SPARK-16116][SQL] ConsoleSink should not require checkpointLocation
## What changes were proposed in this pull request?

When the user uses `ConsoleSink`, we should use a temp location if `checkpointLocation` is not specified.

## How was this patch tested?

The added unit test.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #13817 from zsxwing/console-checkpoint.
2016-06-23 10:46:20 -07:00
Brian Cho 4374a46bfc [SPARK-16162] Remove dead code OrcTableScan.
## What changes were proposed in this pull request?

SPARK-14535 removed all calls to class OrcTableScan. This removes the dead code.

## How was this patch tested?

Existing unit tests.

Author: Brian Cho <bcho@fb.com>

Closes #13869 from dafrista/clean-up-orctablescan.
2016-06-22 22:37:50 -07:00
Cheng Lian f34b5c62b2 [SQL][MINOR] Fix minor formatting issues in SHOW CREATE TABLE output
## What changes were proposed in this pull request?

This PR fixes two minor formatting issues appearing in `SHOW CREATE TABLE` output.

Before:

```
CREATE EXTERNAL TABLE ...
...
WITH SERDEPROPERTIES ('serialization.format' = '1'
)
...
TBLPROPERTIES ('avro.schema.url' = '/tmp/avro/test.avsc',
  'transient_lastDdlTime' = '1466638180')
```

After:

```
CREATE EXTERNAL TABLE ...
...
WITH SERDEPROPERTIES (
  'serialization.format' = '1'
)
...
TBLPROPERTIES (
  'avro.schema.url' = '/tmp/avro/test.avsc',
  'transient_lastDdlTime' = '1466638180'
)
```

## How was this patch tested?

Manually tested.

Author: Cheng Lian <lian@databricks.com>

Closes #13864 from liancheng/show-create-table-format-fix.
2016-06-22 22:28:54 -07:00
bomeng 925884a612 [SPARK-15230][SQL] distinct() does not handle column name with dot properly
## What changes were proposed in this pull request?

When table is created with column name containing dot, distinct() will fail to run. For example,
```scala
val rowRDD = sparkContext.parallelize(Seq(Row(1), Row(1), Row(2)))
val schema = StructType(Array(StructField("column.with.dot", IntegerType, nullable = false)))
val df = spark.createDataFrame(rowRDD, schema)
```
running the following will have no problem:
```scala
df.select(new Column("`column.with.dot`"))
```
but running the query with additional distinct() will cause exception:
```scala
df.select(new Column("`column.with.dot`")).distinct()
```

The issue is that distinct() will try to resolve the column name, but the column name in the schema does not have backtick with it. So the solution is to add the backtick before passing the column name to resolve().

## How was this patch tested?

Added a new test case.

Author: bomeng <bmeng@us.ibm.com>

Closes #13140 from bomeng/SPARK-15230.
2016-06-23 11:06:19 +08:00
Reynold Xin 37f3be5d29 [SPARK-16159][SQL] Move RDD creation logic from FileSourceStrategy.apply
## What changes were proposed in this pull request?
We embed partitioning logic in FileSourceStrategy.apply, making the function very long. This is a small refactoring to move it into its own functions. Eventually we would be able to move the partitioning functions into a physical operator, rather than doing it in physical planning.

## How was this patch tested?
This is a simple code move.

Author: Reynold Xin <rxin@databricks.com>

Closes #13862 from rxin/SPARK-16159.
2016-06-22 18:19:07 -07:00
gatorsmile 9f990fa3f9 [SPARK-16024][SQL][TEST] Verify Column Comment for Data Source Tables
#### What changes were proposed in this pull request?
This PR is to improve test coverage. It verifies whether `Comment` of `Column` can be appropriate handled.

The test cases verify the related parts in Parser, both SQL and DataFrameWriter interface, and both Hive Metastore catalog and In-memory catalog.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13764 from gatorsmile/dataSourceComment.
2016-06-23 09:12:20 +08:00
Brian Cho 4f869f88ee [SPARK-15956][SQL] When unwrapping ORC avoid pattern matching at runtime
## What changes were proposed in this pull request?

Extend the returning of unwrapper functions from primitive types to all types.

This PR is based on https://github.com/apache/spark/pull/13676. It only fixes a bug with scala-2.10 compilation. All credit should go to dafrista.

## How was this patch tested?

The patch should pass all unit tests. Reading ORC files with non-primitive types with this change reduced the read time by ~15%.

Author: Brian Cho <bcho@fb.com>
Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #13854 from hvanhovell/SPARK-15956-scala210.
2016-06-22 16:56:55 -07:00
Davies Liu 20d411bc5d [SPARK-16078][SQL] from_utc_timestamp/to_utc_timestamp should not depends on local timezone
## What changes were proposed in this pull request?

Currently, we use local timezone to parse or format a timestamp (TimestampType), then use Long as the microseconds since epoch UTC.

In from_utc_timestamp() and to_utc_timestamp(), we did not consider the local timezone, they could return different results with different local timezone.

This PR will do the conversion based on human time (in local timezone), it should return same result in whatever timezone. But because the mapping from absolute timestamp to human time is not exactly one-to-one mapping, it will still return wrong result in some timezone (also in the begging or ending of DST).

This PR is kind of the best effort fix. In long term, we should make the TimestampType be timezone aware to fix this totally.

## How was this patch tested?

Tested these function in all timezone.

Author: Davies Liu <davies@databricks.com>

Closes #13784 from davies/convert_tz.
2016-06-22 13:40:24 -07:00
Herman van Hovell 472d611a70 [SPARK-15956][SQL] Revert "[] When unwrapping ORC avoid pattern matching…
This reverts commit 0a9c027595. It breaks the 2.10 build, I'll fix this in a different PR.

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

Closes #13853 from hvanhovell/SPARK-15956-revert.
2016-06-22 11:36:32 -07:00
Brian Cho 0a9c027595 [SPARK-15956][SQL] When unwrapping ORC avoid pattern matching at runtime
## What changes were proposed in this pull request?

Extend the returning of unwrapper functions from primitive types to all types.

## How was this patch tested?

The patch should pass all unit tests. Reading ORC files with non-primitive types with this change reduced the read time by ~15%.

===

The github diff is very noisy. Attaching the screenshots below for improved readability:

![screen shot 2016-06-14 at 5 33 16 pm](https://cloud.githubusercontent.com/assets/1514239/16064580/4d6f7a98-3257-11e6-9172-65e4baff948b.png)

![screen shot 2016-06-14 at 5 33 28 pm](https://cloud.githubusercontent.com/assets/1514239/16064587/5ae6c244-3257-11e6-8460-69eee70de219.png)

Author: Brian Cho <bcho@fb.com>

Closes #13676 from dafrista/improve-orc-master.
2016-06-22 10:38:42 -07:00
Wenchen Fan 01277d4b25 [SPARK-16097][SQL] Encoders.tuple should handle null object correctly
## What changes were proposed in this pull request?

Although the top level input object can not be null, but when we use `Encoders.tuple` to combine 2 encoders, their input objects are not top level anymore and can be null. We should handle this case.

## How was this patch tested?

new test in DatasetSuite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #13807 from cloud-fan/bug.
2016-06-22 18:32:14 +08:00
Yin Huai 39ad53f7ff [SPARK-16121] ListingFileCatalog does not list in parallel anymore
## What changes were proposed in this pull request?
Seems the fix of SPARK-14959 breaks the parallel partitioning discovery. This PR fixes the problem

## How was this patch tested?
Tested manually. (This PR also adds a proper test for SPARK-14959)

Author: Yin Huai <yhuai@databricks.com>

Closes #13830 from yhuai/SPARK-16121.
2016-06-22 18:07:07 +08:00
gatorsmile 0e3ce75332 [SPARK-15644][MLLIB][SQL] Replace SQLContext with SparkSession in MLlib
#### What changes were proposed in this pull request?
This PR is to use the latest `SparkSession` to replace the existing `SQLContext` in `MLlib`. `SQLContext` is removed from `MLlib`.

Also fix a test case issue in `BroadcastJoinSuite`.

BTW, `SQLContext` is not being used in the `MLlib` test suites.
#### How was this patch tested?
Existing test cases.

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

Closes #13380 from gatorsmile/sqlContextML.
2016-06-21 23:12:08 -07:00
hyukjinkwon 7580f3041a [SPARK-16104] [SQL] Do not creaate CSV writer object for every flush when writing
## What changes were proposed in this pull request?

This PR let `CsvWriter` object is not created for each time but able to be reused. This way was taken after from JSON data source.

Original `CsvWriter` was being created for each row but it was enhanced in https://github.com/apache/spark/pull/13229. However, it still creates `CsvWriter` object for each `flush()` in `LineCsvWriter`. It seems it does not have to close the object and re-create this for every flush.

It follows the original logic as it is but `CsvWriter` is reused by reseting `CharArrayWriter`.

## How was this patch tested?

Existing tests should cover this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #13809 from HyukjinKwon/write-perf.
2016-06-21 21:58:38 -07:00
Shixiong Zhu c399c7f0e4 [SPARK-16002][SQL] Sleep when no new data arrives to avoid 100% CPU usage
## What changes were proposed in this pull request?

Add a configuration to allow people to set a minimum polling delay when no new data arrives (default is 10ms). This PR also cleans up some INFO logs.

## How was this patch tested?

Existing unit tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #13718 from zsxwing/SPARK-16002.
2016-06-21 12:42:49 -07:00
Cheng Lian f4a3d45e38 [SPARK-16037][SQL] Follow-up: add DataFrameWriter.insertInto() test cases for by position resolution
## What changes were proposed in this pull request?

This PR migrates some test cases introduced in #12313 as a follow-up of #13754 and #13766. These test cases cover `DataFrameWriter.insertInto()`, while the former two only cover SQL `INSERT` statements.

Note that the `testPartitionedTable` utility method tests both Hive SerDe tables and data source tables.

## How was this patch tested?

N/A

Author: Cheng Lian <lian@databricks.com>

Closes #13810 from liancheng/spark-16037-follow-up-tests.
2016-06-21 11:58:33 -07:00
bomeng f3a768b7b9 [SPARK-16084][SQL] Minor comments update for "DESCRIBE" table
## What changes were proposed in this pull request?

1. FORMATTED is actually supported, but partition is not supported;
2. Remove parenthesis as it is not necessary just like anywhere else.

## How was this patch tested?

Minor issue. I do not think it needs a test case!

Author: bomeng <bmeng@us.ibm.com>

Closes #13791 from bomeng/SPARK-16084.
2016-06-21 08:51:43 +01:00
hyukjinkwon 4f7f1c4362 [SPARK-16044][SQL] input_file_name() returns empty strings in data sources based on NewHadoopRDD
## What changes were proposed in this pull request?

This PR makes `input_file_name()` function return the file paths not empty strings for external data sources based on `NewHadoopRDD`, such as [spark-redshift](cba5eee1ab/src/main/scala/com/databricks/spark/redshift/RedshiftRelation.scala (L149)) and [spark-xml](https://github.com/databricks/spark-xml/blob/master/src/main/scala/com/databricks/spark/xml/util/XmlFile.scala#L39-L47).

The codes with the external data sources below:

```scala
df.select(input_file_name).show()
```

will produce

- **Before**
  ```
+-----------------+
|input_file_name()|
+-----------------+
|                 |
+-----------------+
```

- **After**
  ```
+--------------------+
|   input_file_name()|
+--------------------+
|file:/private/var...|
+--------------------+
```

## How was this patch tested?

Unit tests in `ColumnExpressionSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #13759 from HyukjinKwon/SPARK-16044.
2016-06-20 21:55:34 -07:00
gatorsmile d9a3a2a0be [SPARK-16056][SPARK-16057][SPARK-16058][SQL] Fix Multiple Bugs in Column Partitioning in JDBC Source
#### What changes were proposed in this pull request?
This PR is to fix the following bugs:

**Issue 1: Wrong Results when lowerBound is larger than upperBound in Column Partitioning**
```scala
spark.read.jdbc(
  url = urlWithUserAndPass,
  table = "TEST.seq",
  columnName = "id",
  lowerBound = 4,
  upperBound = 0,
  numPartitions = 3,
  connectionProperties = new Properties)
```
**Before code changes:**
The returned results are wrong and the generated partitions are wrong:
```
  Part 0 id < 3 or id is null
  Part 1 id >= 3 AND id < 2
  Part 2 id >= 2
```
**After code changes:**
Issue an `IllegalArgumentException` exception:
```
Operation not allowed: the lower bound of partitioning column is larger than the upper bound. lowerBound: 5; higherBound: 1
```
**Issue 2: numPartitions is more than the number of key values between upper and lower bounds**
```scala
spark.read.jdbc(
  url = urlWithUserAndPass,
  table = "TEST.seq",
  columnName = "id",
  lowerBound = 1,
  upperBound = 5,
  numPartitions = 10,
  connectionProperties = new Properties)
```
**Before code changes:**
Returned correct results but the generated partitions are very inefficient, like:
```
Partition 0: id < 1 or id is null
Partition 1: id >= 1 AND id < 1
Partition 2: id >= 1 AND id < 1
Partition 3: id >= 1 AND id < 1
Partition 4: id >= 1 AND id < 1
Partition 5: id >= 1 AND id < 1
Partition 6: id >= 1 AND id < 1
Partition 7: id >= 1 AND id < 1
Partition 8: id >= 1 AND id < 1
Partition 9: id >= 1
```
**After code changes:**
Adjust `numPartitions` and can return the correct answers:
```
Partition 0: id < 2 or id is null
Partition 1: id >= 2 AND id < 3
Partition 2: id >= 3 AND id < 4
Partition 3: id >= 4
```
**Issue 3: java.lang.ArithmeticException when numPartitions is zero**
```Scala
spark.read.jdbc(
  url = urlWithUserAndPass,
  table = "TEST.seq",
  columnName = "id",
  lowerBound = 0,
  upperBound = 4,
  numPartitions = 0,
  connectionProperties = new Properties)
```
**Before code changes:**
Got the following exception:
```
  java.lang.ArithmeticException: / by zero
```
**After code changes:**
Able to return a correct answer by disabling column partitioning when numPartitions is equal to or less than zero

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13773 from gatorsmile/jdbcPartitioning.
2016-06-20 21:49:33 -07:00
Reynold Xin c775bf09e0 [SPARK-13792][SQL] Limit logging of bad records in CSV data source
## What changes were proposed in this pull request?
This pull request adds a new option (maxMalformedLogPerPartition) in CSV reader to limit the maximum of logging message Spark generates per partition for malformed records.

The error log looks something like
```
16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4
16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4
16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4
16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4
16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4
16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4
16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4
16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4
16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4
16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4
16/06/20 18:50:14 WARN CSVRelation: More than 10 malformed records have been found on this partition. Malformed records from now on will not be logged.
```

Closes #12173

## How was this patch tested?
Manually tested.

Author: Reynold Xin <rxin@databricks.com>

Closes #13795 from rxin/SPARK-13792.
2016-06-20 21:46:12 -07:00
Kousuke Saruta 6daa8cf1a6 [SPARK-16061][SQL][MINOR] The property "spark.streaming.stateStore.maintenanceInterval" should be renamed to "spark.sql.streaming.stateStore.maintenanceInterval"
## What changes were proposed in this pull request?
The property spark.streaming.stateStore.maintenanceInterval should be renamed and harmonized with other properties related to Structured Streaming like spark.sql.streaming.stateStore.minDeltasForSnapshot.

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

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

Closes #13777 from sarutak/SPARK-16061.
2016-06-20 15:12:40 -07:00
Tathagata Das b99129cc45 [SPARK-15982][SPARK-16009][SPARK-16007][SQL] Harmonize the behavior of DataFrameReader.text/csv/json/parquet/orc
## What changes were proposed in this pull request?

Issues with current reader behavior.
- `text()` without args returns an empty DF with no columns -> inconsistent, its expected that text will always return a DF with `value` string field,
- `textFile()` without args fails with exception because of the above reason, it expected the DF returned by `text()` to have a `value` field.
- `orc()` does not have var args, inconsistent with others
- `json(single-arg)` was removed, but that caused source compatibility issues - [SPARK-16009](https://issues.apache.org/jira/browse/SPARK-16009)
- user specified schema was not respected when `text/csv/...` were used with no args - [SPARK-16007](https://issues.apache.org/jira/browse/SPARK-16007)

The solution I am implementing is to do the following.
- For each format, there will be a single argument method, and a vararg method. For json, parquet, csv, text, this means adding json(string), etc.. For orc, this means adding orc(varargs).
- Remove the special handling of text(), csv(), etc. that returns empty dataframe with no fields. Rather pass on the empty sequence of paths to the datasource, and let each datasource handle it right. For e.g, text data source, should return empty DF with schema (value: string)
- Deduped docs and fixed their formatting.

## How was this patch tested?
Added new unit tests for Scala and Java tests

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

Closes #13727 from tdas/SPARK-15982.
2016-06-20 14:52:28 -07:00
Shixiong Zhu 5cfabec872 [SPARK-16050][TESTS] Remove the flaky test: ConsoleSinkSuite
## What changes were proposed in this pull request?

ConsoleSinkSuite just collects content from stdout and compare them with the expected string. However, because Spark may not stop some background threads at once, there is a race condition that other threads are outputting logs to **stdout** while ConsoleSinkSuite is running. Then it will make ConsoleSinkSuite fail.

Therefore, I just deleted `ConsoleSinkSuite`. If we want to test ConsoleSinkSuite in future, we should refactoring ConsoleSink to make it testable instead of depending on stdout. Therefore, this test is useless and I just delete it.

## How was this patch tested?

Just removed a flaky test.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #13776 from zsxwing/SPARK-16050.
2016-06-20 10:35:37 -07:00
Yin Huai 905f774b71 [SPARK-16030][SQL] Allow specifying static partitions when inserting to data source tables
## What changes were proposed in this pull request?
This PR adds the static partition support to INSERT statement when the target table is a data source table.

## How was this patch tested?
New tests in InsertIntoHiveTableSuite and DataSourceAnalysisSuite.

**Note: This PR is based on https://github.com/apache/spark/pull/13766. The last commit is the actual change.**

Author: Yin Huai <yhuai@databricks.com>

Closes #13769 from yhuai/SPARK-16030-1.
2016-06-20 20:17:47 +08:00
Yin Huai 6d0f921aed [SPARK-16036][SPARK-16037][SPARK-16034][SQL] Follow up code clean up and improvement
## What changes were proposed in this pull request?
This PR is the follow-up PR for https://github.com/apache/spark/pull/13754/files and https://github.com/apache/spark/pull/13749. I will comment inline to explain my changes.

## How was this patch tested?
Existing tests.

Author: Yin Huai <yhuai@databricks.com>

Closes #13766 from yhuai/caseSensitivity.
2016-06-19 21:45:53 -07:00
Matei Zaharia 4f17fddcd5 [SPARK-16031] Add debug-only socket source in Structured Streaming
## What changes were proposed in this pull request?

This patch adds a text-based socket source similar to the one in Spark Streaming for debugging and tutorials. The source is clearly marked as debug-only so that users don't try to run it in production applications, because this type of source cannot provide HA without storing a lot of state in Spark.

## How was this patch tested?

Unit tests and manual tests in spark-shell.

Author: Matei Zaharia <matei@databricks.com>

Closes #13748 from mateiz/socket-source.
2016-06-19 21:27:04 -07:00
Davies Liu 001a589603 [SPARK-15613] [SQL] Fix incorrect days to millis conversion due to Daylight Saving Time
## What changes were proposed in this pull request?

Internally, we use Int to represent a date (the days since 1970-01-01), when we convert that into unix timestamp (milli-seconds since epoch in UTC), we get the offset of a timezone using local millis (the milli-seconds since 1970-01-01 in a timezone), but TimeZone.getOffset() expect unix timestamp, the result could be off by one hour (in Daylight Saving Time (DST) or not).

This PR change to use best effort approximate of posix timestamp to lookup the offset. In the event of changing of DST, Some time is not defined (for example, 2016-03-13 02:00:00 PST), or could lead to multiple valid result in UTC (for example, 2016-11-06 01:00:00), this best effort approximate should be enough in practice.

## How was this patch tested?

Added regression tests.

Author: Davies Liu <davies@databricks.com>

Closes #13652 from davies/fix_timezone.
2016-06-19 00:34:52 -07:00
Sean Zhong ce3b98bae2 [SPARK-16034][SQL] Checks the partition columns when calling dataFrame.write.mode("append").saveAsTable
## What changes were proposed in this pull request?

`DataFrameWriter` can be used to append data to existing data source tables. It becomes tricky when partition columns used in `DataFrameWriter.partitionBy(columns)` don't match the actual partition columns of the underlying table. This pull request enforces the check so that the partition columns of these two always match.

## How was this patch tested?

Unit test.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #13749 from clockfly/SPARK-16034.
2016-06-18 10:41:33 -07:00
Wenchen Fan 3d010c8375 [SPARK-16036][SPARK-16037][SQL] fix various table insertion problems
## What changes were proposed in this pull request?

The current table insertion has some weird behaviours:

1. inserting into a partitioned table with mismatch columns has confusing error message for hive table, and wrong result for datasource table
2. inserting into a partitioned table without partition list has wrong result for hive table.

This PR fixes these 2 problems.

## How was this patch tested?

new test in hive `SQLQuerySuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #13754 from cloud-fan/insert2.
2016-06-18 10:32:27 -07:00
Andrew Or 35a2f3c012 [SPARK-16023][SQL] Move InMemoryRelation to its own file
## What changes were proposed in this pull request?

Improve readability of `InMemoryTableScanExec.scala`, which has too much stuff in it.

## How was this patch tested?

Jenkins

Author: Andrew Or <andrew@databricks.com>

Closes #13742 from andrewor14/move-inmemory-relation.
2016-06-17 23:41:09 -07:00
Shixiong Zhu d0ac0e6f43 [SPARK-16020][SQL] Fix complete mode aggregation with console sink
## What changes were proposed in this pull request?

We cannot use `limit` on DataFrame in ConsoleSink because it will use a wrong planner. This PR just collects `DataFrame` and calls `show` on a batch DataFrame based on the result. This is fine since ConsoleSink is only for debugging.

## How was this patch tested?

Manually confirmed ConsoleSink now works with complete mode aggregation.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #13740 from zsxwing/complete-console.
2016-06-17 21:58:10 -07:00
Felix Cheung 8c198e246d [SPARK-15159][SPARKR] SparkR SparkSession API
## What changes were proposed in this pull request?

This PR introduces the new SparkSession API for SparkR.
`sparkR.session.getOrCreate()` and `sparkR.session.stop()`

"getOrCreate" is a bit unusual in R but it's important to name this clearly.

SparkR implementation should
- SparkSession is the main entrypoint (vs SparkContext; due to limited functionality supported with SparkContext in SparkR)
- SparkSession replaces SQLContext and HiveContext (both a wrapper around SparkSession, and because of API changes, supporting all 3 would be a lot more work)
- Changes to SparkSession is mostly transparent to users due to SPARK-10903
- Full backward compatibility is expected - users should be able to initialize everything just in Spark 1.6.1 (`sparkR.init()`), but with deprecation warning
- Mostly cosmetic changes to parameter list - users should be able to move to `sparkR.session.getOrCreate()` easily
- An advanced syntax with named parameters (aka varargs aka "...") is supported; that should be closer to the Builder syntax that is in Scala/Python (which unfortunately does not work in R because it will look like this: `enableHiveSupport(config(config(master(appName(builder(), "foo"), "local"), "first", "value"), "next, "value"))`
- Updating config on an existing SparkSession is supported, the behavior is the same as Python, in which config is applied to both SparkContext and SparkSession
- Some SparkSession changes are not matched in SparkR, mostly because it would be breaking API change: `catalog` object, `createOrReplaceTempView`
- Other SQLContext workarounds are replicated in SparkR, eg. `tables`, `tableNames`
- `sparkR` shell is updated to use the SparkSession entrypoint (`sqlContext` is removed, just like with Scale/Python)
- All tests are updated to use the SparkSession entrypoint
- A bug in `read.jdbc` is fixed

TODO
- [x] Add more tests
- [ ] Separate PR - update all roxygen2 doc coding example
- [ ] Separate PR - update SparkR programming guide

## How was this patch tested?

unit tests, manual tests

shivaram sun-rui rxin

Author: Felix Cheung <felixcheung_m@hotmail.com>
Author: felixcheung <felixcheung_m@hotmail.com>

Closes #13635 from felixcheung/rsparksession.
2016-06-17 21:36:01 -07:00
Cheng Lian 10b671447b [SPARK-16033][SQL] insertInto() can't be used together with partitionBy()
## What changes were proposed in this pull request?

When inserting into an existing partitioned table, partitioning columns should always be determined by catalog metadata of the existing table to be inserted. Extra `partitionBy()` calls don't make sense, and mess up existing data because newly inserted data may have wrong partitioning directory layout.

## How was this patch tested?

New test case added in `InsertIntoHiveTableSuite`.

Author: Cheng Lian <lian@databricks.com>

Closes #13747 from liancheng/spark-16033-insert-into-without-partition-by.
2016-06-17 20:13:04 -07:00
hyukjinkwon ebb9a3b6fd [SPARK-15916][SQL] JDBC filter push down should respect operator precedence
## What changes were proposed in this pull request?

This PR fixes the problem that the precedence order is messed when pushing where-clause expression to JDBC layer.

**Case 1:**

For sql `select * from table where (a or b) and c`, the where-clause is wrongly converted to JDBC where-clause `a or (b and c)` after filter push down. The consequence is that JDBC may returns less or more rows than expected.

**Case 2:**

For sql `select * from table where always_false_condition`, the result table may not be empty if the JDBC RDD is partitioned using where-clause:
```
spark.read.jdbc(url, table, predicates = Array("partition 1 where clause", "partition 2 where clause"...)
```

## How was this patch tested?

Unit test.

This PR also close #13640

Author: hyukjinkwon <gurwls223@gmail.com>
Author: Sean Zhong <seanzhong@databricks.com>

Closes #13743 from clockfly/SPARK-15916.
2016-06-17 17:11:38 -07:00
Reynold Xin 1a65e62a7f [SPARK-16014][SQL] Rename optimizer rules to be more consistent
## What changes were proposed in this pull request?
This small patch renames a few optimizer rules to make the naming more consistent, e.g. class name start with a verb. The main important "fix" is probably SamplePushDown -> PushProjectThroughSample. SamplePushDown is actually the wrong name, since the rule is not about pushing Sample down.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #13732 from rxin/SPARK-16014.
2016-06-17 15:51:20 -07:00
Sameer Agarwal 34d6c4cd11 Remove non-obvious conf settings from TPCDS benchmark
## What changes were proposed in this pull request?

My fault -- these 2 conf entries are mysteriously hidden inside the benchmark code and makes it non-obvious to disable whole stage codegen and/or the vectorized parquet reader.

PS: Didn't attach a JIRA as this change should otherwise be a no-op (both these conf are enabled by default in Spark)

## How was this patch tested?

N/A

Author: Sameer Agarwal <sameer@databricks.com>

Closes #13726 from sameeragarwal/tpcds-conf.
2016-06-17 09:47:41 -07:00
Davies Liu ef43b4ed87 [SPARK-15811][SQL] fix the Python UDF in Scala 2.10
## What changes were proposed in this pull request?

Iterator can't be serialized in Scala 2.10, we should force it into a array to make sure that .

## How was this patch tested?

Build with Scala 2.10 and ran all the Python unit tests manually (will be covered by a jenkins build).

Author: Davies Liu <davies@databricks.com>

Closes #13717 from davies/fix_udf_210.
2016-06-17 00:34:33 -07:00
gatorsmile e5d703bca8 [SPARK-15706][SQL] Fix Wrong Answer when using IF NOT EXISTS in INSERT OVERWRITE for DYNAMIC PARTITION
#### What changes were proposed in this pull request?
`IF NOT EXISTS` in `INSERT OVERWRITE` should not support dynamic partitions. If we specify `IF NOT EXISTS`, the inserted statement is not shown in the table.

This PR is to issue an exception in this case, just like what Hive does. Also issue an exception if users specify `IF NOT EXISTS` if users do not specify any `PARTITION` specification.

#### How was this patch tested?
Added test cases into `PlanParserSuite` and `InsertIntoHiveTableSuite`

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13447 from gatorsmile/insertIfNotExist.
2016-06-16 22:54:02 -07:00
Pete Robbins 5ada606144 [SPARK-15822] [SQL] Prevent byte array backed classes from referencing freed memory
## What changes were proposed in this pull request?
`UTF8String` and all `Unsafe*` classes are backed by either on-heap or off-heap byte arrays. The code generated version `SortMergeJoin` buffers the left hand side join keys during iteration. This was actually problematic in off-heap mode when one of the keys is a `UTF8String` (or any other 'Unsafe*` object) and the left hand side iterator was exhausted (and released its memory); the buffered keys would reference freed memory. This causes Seg-faults and all kinds of other undefined behavior when we would use one these buffered keys.

This PR fixes this problem by creating copies of the buffered variables. I have added a general method to the `CodeGenerator` for this. I have checked all places in which this could happen, and only `SortMergeJoin` had this problem.

This PR is largely based on the work of robbinspg and he should be credited for this.

closes https://github.com/apache/spark/pull/13707

## How was this patch tested?
Manually tested on problematic workloads.

Author: Pete Robbins <robbinspg@gmail.com>
Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #13723 from hvanhovell/SPARK-15822-2.
2016-06-16 22:27:32 -07:00
Yin Huai d9c6628c47 [SPARK-15991] SparkContext.hadoopConfiguration should be always the base of hadoop conf created by SessionState
## What changes were proposed in this pull request?
Before this patch, after a SparkSession has been created, hadoop conf set directly to SparkContext.hadoopConfiguration will not affect the hadoop conf created by SessionState. This patch makes the change to always use SparkContext.hadoopConfiguration  as the base.

This patch also changes the behavior of hive-site.xml support added in https://github.com/apache/spark/pull/12689/. With this patch, we will load hive-site.xml to SparkContext.hadoopConfiguration.

## How was this patch tested?
New test in SparkSessionBuilderSuite.

Author: Yin Huai <yhuai@databricks.com>

Closes #13711 from yhuai/SPARK-15991.
2016-06-16 17:06:24 -07:00
Huaxin Gao 62d2fa5e99 [SPARK-15749][SQL] make the error message more meaningful
## What changes were proposed in this pull request?

For table test1 (C1 varchar (10), C2 varchar (10)), when I insert a row using
```
sqlContext.sql("insert into test1 values ('abc', 'def', 1)")
```
I got error message

```
Exception in thread "main" java.lang.RuntimeException: RelationC1#0,C2#1 JDBCRelation(test1)
requires that the query in the SELECT clause of the INSERT INTO/OVERWRITE statement
generates the same number of columns as its schema.
```
The error message is a little confusing. In my simple insert statement, it doesn't have a SELECT clause.

I will change the error message to a more general one

```
Exception in thread "main" java.lang.RuntimeException: RelationC1#0,C2#1 JDBCRelation(test1)
requires that the data to be inserted have the same number of columns as the target table.
```

## How was this patch tested?

I tested the patch using my simple unit test, but it's a very trivial change and I don't think I need to check in any test.

Author: Huaxin Gao <huaxing@us.ibm.com>

Closes #13492 from huaxingao/spark-15749.
2016-06-16 14:37:10 -07:00
Dongjoon Hyun 2d27eb1e75 [MINOR][DOCS][SQL] Fix some comments about types(TypeCoercion,Partition) and exceptions.
## What changes were proposed in this pull request?

This PR contains a few changes on code comments.
- `HiveTypeCoercion` is renamed into `TypeCoercion`.
- `NoSuchDatabaseException` is only used for the absence of database.
- For partition type inference, only `DoubleType` is considered.

## How was this patch tested?

N/A

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13674 from dongjoon-hyun/minor_doc_types.
2016-06-16 14:27:09 -07:00
gatorsmile 796429d711 [SPARK-15998][SQL] Verification of SQLConf HIVE_METASTORE_PARTITION_PRUNING
#### What changes were proposed in this pull request?
`HIVE_METASTORE_PARTITION_PRUNING` is a public `SQLConf`. When `true`, some predicates will be pushed down into the Hive metastore so that unmatching partitions can be eliminated earlier. The current default value is `false`. For performance improvement, users might turn this parameter on.

So far, the code base does not have such a test case to verify whether this `SQLConf` properly works. This PR is to improve the test case coverage for avoiding future regression.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13716 from gatorsmile/addTestMetastorePartitionPruning.
2016-06-16 14:23:17 -07:00
Cheng Lian 7a89f2adbb [SQL] Minor HashAggregateExec string output fixes
## What changes were proposed in this pull request?

This PR fixes some minor `.toString` format issues for `HashAggregateExec`.

Before:

```
*HashAggregate(key=[a#234L,b#235L], functions=[count(1),max(c#236L)], output=[a#234L,b#235L,count(c)#247L,max(c)#248L])
```

After:

```
*HashAggregate(keys=[a#234L, b#235L], functions=[count(1), max(c#236L)], output=[a#234L, b#235L, count(c)#247L, max(c)#248L])
```

## How was this patch tested?

Manually tested.

Author: Cheng Lian <lian@databricks.com>

Closes #13710 from liancheng/minor-agg-string-fix.
2016-06-16 14:20:44 -07:00
bomeng bbad4cb48d [SPARK-15978][SQL] improve 'show tables' command related codes
## What changes were proposed in this pull request?

I've found some minor issues in "show tables" command:

1. In the `SessionCatalog.scala`, `listTables(db: String)` method will call `listTables(formatDatabaseName(db), "*")` to list all the tables for certain db, but in the method `listTables(db: String, pattern: String)`, this db name is formatted once more. So I think we should remove
`formatDatabaseName()` in the caller.

2. I suggest to add sort to listTables(db: String) in InMemoryCatalog.scala, just like listDatabases().

## How was this patch tested?

The existing test cases should cover it.

Author: bomeng <bmeng@us.ibm.com>

Closes #13695 from bomeng/SPARK-15978.
2016-06-16 14:18:02 -07:00
Herman van Hovell f9bf15d9bd [SPARK-15977][SQL] Fix TRUNCATE TABLE for Spark specific datasource tables
## What changes were proposed in this pull request?
`TRUNCATE TABLE` is currently broken for Spark specific datasource tables (json, csv, ...). This PR correctly sets the location for these datasources which allows them to be truncated.

## How was this patch tested?
Extended the datasources `TRUNCATE TABLE` tests in `DDLSuite`.

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

Closes #13697 from hvanhovell/SPARK-15977.
2016-06-16 13:47:36 -07:00
Cheng Lian 9ea0d5e326 [SPARK-15983][SQL] Removes FileFormat.prepareRead
## What changes were proposed in this pull request?

Interface method `FileFormat.prepareRead()` was added in #12088 to handle a special case in the LibSVM data source.

However, the semantics of this interface method isn't intuitive: it returns a modified version of the data source options map. Considering that the LibSVM case can be easily handled using schema metadata inside `inferSchema`, we can remove this interface method to keep the `FileFormat` interface clean.

## How was this patch tested?

Existing tests.

Author: Cheng Lian <lian@databricks.com>

Closes #13698 from liancheng/remove-prepare-read.
2016-06-16 10:24:29 -07:00
gatorsmile 6451cf9270 [SPARK-15862][SQL] Better Error Message When Having Database Name in CACHE TABLE AS SELECT
#### What changes were proposed in this pull request?
~~If the temp table already exists, we should not silently replace it when doing `CACHE TABLE AS SELECT`. This is inconsistent with the behavior of `CREAT VIEW` or `CREATE TABLE`. This PR is to fix this silent drop.~~

~~Maybe, we also can introduce new syntax for replacing the existing one. For example, in Hive, to replace a view, the syntax should be like `ALTER VIEW AS SELECT` or `CREATE OR REPLACE VIEW AS SELECT`~~

The table name in `CACHE TABLE AS SELECT` should NOT contain database prefix like "database.table". Thus, this PR captures this in Parser and outputs a better error message, instead of reporting the view already exists.

In addition, refactoring the `Parser` to generate table identifiers instead of returning the table name string.

#### How was this patch tested?
- Added a test case for caching and uncaching qualified table names
- Fixed a few test cases that do not drop temp table at the end
- Added the related test case for the issue resolved in this PR

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

Closes #13572 from gatorsmile/cacheTableAsSelect.
2016-06-16 10:01:59 -07:00
Narine Kokhlikyan 7c6c692637 [SPARK-12922][SPARKR][WIP] Implement gapply() on DataFrame in SparkR
## What changes were proposed in this pull request?

gapply() applies an R function on groups grouped by one or more columns of a DataFrame, and returns a DataFrame. It is like GroupedDataSet.flatMapGroups() in the Dataset API.

Please, let me know what do you think and if you have any ideas to improve it.

Thank you!

## How was this patch tested?
Unit tests.
1. Primitive test with different column types
2. Add a boolean column
3. Compute average by a group

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

Closes #12836 from NarineK/gapply2.
2016-06-15 21:42:05 -07:00
Herman van Hovell b75f454f94 [SPARK-15824][SQL] Execute WITH .... INSERT ... statements immediately
## What changes were proposed in this pull request?
We currently immediately execute `INSERT` commands when they are issued. This is not the case as soon as we use a `WITH` to define common table expressions, for example:
```sql
WITH
tbl AS (SELECT * FROM x WHERE id = 10)
INSERT INTO y
SELECT *
FROM   tbl
```

This PR fixes this problem. This PR closes https://github.com/apache/spark/pull/13561 (which fixes the a instance of this problem in the ThriftSever).

## How was this patch tested?
Added a test to `InsertSuite`

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

Closes #13678 from hvanhovell/SPARK-15824.
2016-06-15 21:33:26 -07:00
Wayne Song ebdd751272 [SPARK-13498][SQL] Increment the recordsRead input metric for JDBC data source
## What changes were proposed in this pull request?
This patch brings https://github.com/apache/spark/pull/11373 up-to-date and increments the record count for JDBC data source.

Closes #11373.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #13694 from rxin/SPARK-13498.
2016-06-15 20:09:47 -07:00
Reynold Xin 865e7cc38d [SPARK-15979][SQL] Rename various Parquet support classes.
## What changes were proposed in this pull request?
This patch renames various Parquet support classes from CatalystAbc to ParquetAbc. This new naming makes more sense for two reasons:

1. These are not optimizer related (i.e. Catalyst) classes.
2. We are in the Spark code base, and as a result it'd be more clear to call out these are Parquet support classes, rather than some Spark classes.

## How was this patch tested?
Renamed test cases as well.

Author: Reynold Xin <rxin@databricks.com>

Closes #13696 from rxin/parquet-rename.
2016-06-15 20:05:08 -07:00
KaiXinXiaoLei 3e6d567a46 [SPARK-12492][SQL] Add missing SQLExecution.withNewExecutionId for hiveResultString
## What changes were proposed in this pull request?

Add missing SQLExecution.withNewExecutionId for hiveResultString so that queries running in `spark-sql` will be shown in Web UI.

Closes #13115

## How was this patch tested?

Existing unit tests.

Author: KaiXinXiaoLei <huleilei1@huawei.com>

Closes #13689 from zsxwing/pr13115.
2016-06-15 16:11:46 -07:00
Sean Zhong 9bd80ad6bd [SPARK-15776][SQL] Divide Expression inside Aggregation function is casted to wrong type
## What changes were proposed in this pull request?

This PR fixes the problem that Divide Expression inside Aggregation function is casted to wrong type, which cause `select 1/2` and `select sum(1/2)`returning different result.

**Before the change:**

```
scala> sql("select 1/2 as a").show()
+---+
|  a|
+---+
|0.5|
+---+

scala> sql("select sum(1/2) as a").show()
+---+
|  a|
+---+
|0  |
+---+

scala> sql("select sum(1 / 2) as a").schema
res4: org.apache.spark.sql.types.StructType = StructType(StructField(a,LongType,true))
```

**After the change:**

```
scala> sql("select 1/2 as a").show()
+---+
|  a|
+---+
|0.5|
+---+

scala> sql("select sum(1/2) as a").show()
+---+
|  a|
+---+
|0.5|
+---+

scala> sql("select sum(1/2) as a").schema
res4: org.apache.spark.sql.types.StructType = StructType(StructField(a,DoubleType,true))
```

## How was this patch tested?

Unit test.

This PR is based on https://github.com/apache/spark/pull/13524 by Sephiroth-Lin

Author: Sean Zhong <seanzhong@databricks.com>

Closes #13651 from clockfly/SPARK-15776.
2016-06-15 14:34:15 -07:00
Egor Pakhomov 049e639fc2 [SPARK-15934] [SQL] Return binary mode in ThriftServer
Returning binary mode to ThriftServer for backward compatibility.

Tested with Squirrel and Tableau.

Author: Egor Pakhomov <egor@anchorfree.com>

Closes #13667 from epahomov/SPARK-15095-2.0.
2016-06-15 14:29:32 -07:00
gatorsmile 09925735b5 [SPARK-15901][SQL][TEST] Verification of CONVERT_METASTORE_ORC and CONVERT_METASTORE_PARQUET
#### What changes were proposed in this pull request?
So far, we do not have test cases for verifying whether the external parameters `HiveUtils .CONVERT_METASTORE_ORC` and `HiveUtils.CONVERT_METASTORE_PARQUET` properly works when users use non-default values. This PR is to add such test cases for avoiding potential regression.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13622 from gatorsmile/addTestCase4parquetOrcConversion.
2016-06-15 14:08:55 -07:00
Davies Liu 5389013acc [SPARK-15888] [SQL] fix Python UDF with aggregate
## What changes were proposed in this pull request?

After we move the ExtractPythonUDF rule into physical plan, Python UDF can't work on top of aggregate anymore, because they can't be evaluated before aggregate, should be evaluated after aggregate. This PR add another rule to extract these kind of Python UDF from logical aggregate, create a Project on top of Aggregate.

## How was this patch tested?

Added regression tests. The plan of added test query looks like this:
```
== Parsed Logical Plan ==
'Project [<lambda>('k, 's) AS t#26]
+- Aggregate [<lambda>(key#5L)], [<lambda>(key#5L) AS k#17, sum(cast(<lambda>(value#6) as bigint)) AS s#22L]
   +- LogicalRDD [key#5L, value#6]

== Analyzed Logical Plan ==
t: int
Project [<lambda>(k#17, s#22L) AS t#26]
+- Aggregate [<lambda>(key#5L)], [<lambda>(key#5L) AS k#17, sum(cast(<lambda>(value#6) as bigint)) AS s#22L]
   +- LogicalRDD [key#5L, value#6]

== Optimized Logical Plan ==
Project [<lambda>(agg#29, agg#30L) AS t#26]
+- Aggregate [<lambda>(key#5L)], [<lambda>(key#5L) AS agg#29, sum(cast(<lambda>(value#6) as bigint)) AS agg#30L]
   +- LogicalRDD [key#5L, value#6]

== Physical Plan ==
*Project [pythonUDF0#37 AS t#26]
+- BatchEvalPython [<lambda>(agg#29, agg#30L)], [agg#29, agg#30L, pythonUDF0#37]
   +- *HashAggregate(key=[<lambda>(key#5L)#31], functions=[sum(cast(<lambda>(value#6) as bigint))], output=[agg#29,agg#30L])
      +- Exchange hashpartitioning(<lambda>(key#5L)#31, 200)
         +- *HashAggregate(key=[pythonUDF0#34 AS <lambda>(key#5L)#31], functions=[partial_sum(cast(pythonUDF1#35 as bigint))], output=[<lambda>(key#5L)#31,sum#33L])
            +- BatchEvalPython [<lambda>(key#5L), <lambda>(value#6)], [key#5L, value#6, pythonUDF0#34, pythonUDF1#35]
               +- Scan ExistingRDD[key#5L,value#6]
```

Author: Davies Liu <davies@databricks.com>

Closes #13682 from davies/fix_py_udf.
2016-06-15 13:38:04 -07:00
Yin Huai e1585cc748 [SPARK-15959][SQL] Add the support of hive.metastore.warehouse.dir back
## What changes were proposed in this pull request?
This PR adds the support of conf `hive.metastore.warehouse.dir` back. With this patch, the way of setting the warehouse dir is described as follows:
* If `spark.sql.warehouse.dir` is set, `hive.metastore.warehouse.dir` will be automatically set to the value of `spark.sql.warehouse.dir`. The warehouse dir is effectively set to the value of `spark.sql.warehouse.dir`.
* If `spark.sql.warehouse.dir` is not set but `hive.metastore.warehouse.dir` is set, `spark.sql.warehouse.dir` will be automatically set to the value of `hive.metastore.warehouse.dir`. The warehouse dir is effectively set to the value of `hive.metastore.warehouse.dir`.
* If neither `spark.sql.warehouse.dir` nor `hive.metastore.warehouse.dir` is set, `hive.metastore.warehouse.dir` will be automatically set to the default value of `spark.sql.warehouse.dir`. The warehouse dir is effectively set to the default value of `spark.sql.warehouse.dir`.

## How was this patch tested?
`set hive.metastore.warehouse.dir` in `HiveSparkSubmitSuite`.

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

Author: Yin Huai <yhuai@databricks.com>

Closes #13679 from yhuai/hiveWarehouseDir.
2016-06-15 11:50:54 -07:00
Tathagata Das 9a5071996b [SPARK-15953][WIP][STREAMING] Renamed ContinuousQuery to StreamingQuery
Renamed for simplicity, so that its obvious that its related to streaming.

Existing unit tests.

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

Closes #13673 from tdas/SPARK-15953.
2016-06-15 10:46:07 -07:00
Herman van Hovell de99c3d081 [SPARK-15960][SQL] Rename spark.sql.enableFallBackToHdfsForStats config
## What changes were proposed in this pull request?
Since we are probably going to add more statistics related configurations in the future, I'd like to rename the newly added `spark.sql.enableFallBackToHdfsForStats` configuration option to `spark.sql.statistics.fallBackToHdfs`. This allows us to put all statistics related configurations in the same namespace.

## How was this patch tested?
None - just a usability thing

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

Closes #13681 from hvanhovell/SPARK-15960.
2016-06-15 09:43:11 -07:00
bomeng 42a28caf10 [SPARK-15952][SQL] fix "show databases" ordering issue
## What changes were proposed in this pull request?

Two issues I've found for "show databases" command:

1. The returned database name list was not sorted, it only works when "like" was used together; (HIVE will always return a sorted list)

2. When it is used as sql("show databases").show, it will output a table with column named as "result", but for sql("show tables").show, it will output the column name as "tableName", so I think we should be consistent and use "databaseName" at least.

## How was this patch tested?

Updated existing test case to test its ordering as well.

Author: bomeng <bmeng@us.ibm.com>

Closes #13671 from bomeng/SPARK-15952.
2016-06-14 18:35:29 -07:00
Herman van Hovell 0bd86c0fe4 [SPARK-15011][SQL] Re-enable 'analyze MetastoreRelations' in hive StatisticsSuite
## What changes were proposed in this pull request?
This test re-enables the `analyze MetastoreRelations` in `org.apache.spark.sql.hive.StatisticsSuite`.

The flakiness of this test was traced back to a shared configuration option, `hive.exec.compress.output`, in `TestHive`. This property was set to `true` by the `HiveCompatibilitySuite`. I have added configuration resetting logic to `HiveComparisonTest`, in order to prevent such a thing from happening again.

## How was this patch tested?
Is a test.

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

Closes #13498 from hvanhovell/SPARK-15011.
2016-06-14 18:24:59 -07:00
Tathagata Das 214adb14b8 [SPARK-15933][SQL][STREAMING] Refactored DF reader-writer to use readStream and writeStream for streaming DFs
## What changes were proposed in this pull request?
Currently, the DataFrameReader/Writer has method that are needed for streaming and non-streaming DFs. This is quite awkward because each method in them through runtime exception for one case or the other. So rather having half the methods throw runtime exceptions, its just better to have a different reader/writer API for streams.

- [x] Python API!!

## How was this patch tested?
Existing unit tests + two sets of unit tests for DataFrameReader/Writer and DataStreamReader/Writer.

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

Closes #13653 from tdas/SPARK-15933.
2016-06-14 17:58:45 -07:00
Takeshi YAMAMURO dae4d5db21 [SPARK-15247][SQL] Set the default number of partitions for reading parquet schemas
## What changes were proposed in this pull request?
This pr sets the default number of partitions when reading parquet schemas.
SQLContext#read#parquet currently yields at least n_executors * n_cores tasks even if parquet data consist of a  single small file. This issue could increase the latency for small jobs.

## How was this patch tested?
Manually tested and checked.

Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes #13137 from maropu/SPARK-15247.
2016-06-14 13:05:56 -07:00
Cheng Lian bd39ffe35c [SPARK-15895][SQL] Filters out metadata files while doing partition discovery
## What changes were proposed in this pull request?

Take the following directory layout as an example:

```
dir/
+- p0=0/
   |-_metadata
   +- p1=0/
      |-part-00001.parquet
      |-part-00002.parquet
      |-...
```

The `_metadata` file under `p0=0` shouldn't fail partition discovery.

This PR filters output all metadata files whose names start with `_` while doing partition discovery.

## How was this patch tested?

New unit test added in `ParquetPartitionDiscoverySuite`.

Author: Cheng Lian <lian@databricks.com>

Closes #13623 from liancheng/spark-15895-partition-disco-no-metafiles.
2016-06-14 12:13:12 -07:00
gatorsmile df4ea6614d [SPARK-15864][SQL] Fix Inconsistent Behaviors when Uncaching Non-cached Tables
#### What changes were proposed in this pull request?
To uncache a table, we have three different ways:
- _SQL interface_: `UNCACHE TABLE`
- _DataSet API_: `sparkSession.catalog.uncacheTable`
- _DataSet API_: `sparkSession.table(tableName).unpersist()`

When the table is not cached,
- _SQL interface_: `UNCACHE TABLE non-cachedTable` -> **no error message**
- _Dataset API_: `sparkSession.catalog.uncacheTable("non-cachedTable")` -> **report a strange error message:**
```requirement failed: Table [a: int] is not cached```
- _Dataset API_: `sparkSession.table("non-cachedTable").unpersist()` -> **no error message**

This PR will make them consistent. No operation if the table has already been uncached.

In addition, this PR also removes `uncacheQuery` and renames `tryUncacheQuery` to `uncacheQuery`, and documents it that it's noop if the table has already been uncached

#### How was this patch tested?
Improved the existing test case for verifying the cases when the table has not been cached.
Also added test cases for verifying the cases when the table does not exist

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

Closes #13593 from gatorsmile/uncacheNonCachedTable.
2016-06-14 11:44:37 -07:00
Takuya UESHIN c5b7355819 [SPARK-15915][SQL] Logical plans should use canonicalized plan when override sameResult.
## What changes were proposed in this pull request?

`DataFrame` with plan overriding `sameResult` but not using canonicalized plan to compare can't cacheTable.

The example is like:

```
    val localRelation = Seq(1, 2, 3).toDF()
    localRelation.createOrReplaceTempView("localRelation")

    spark.catalog.cacheTable("localRelation")
    assert(
      localRelation.queryExecution.withCachedData.collect {
        case i: InMemoryRelation => i
      }.size == 1)
```

and this will fail as:

```
ArrayBuffer() had size 0 instead of expected size 1
```

The reason is that when do `spark.catalog.cacheTable("localRelation")`, `CacheManager` tries to cache for the plan wrapped by `SubqueryAlias` but when planning for the DataFrame `localRelation`, `CacheManager` tries to find cached table for the not-wrapped plan because the plan for DataFrame `localRelation` is not wrapped.
Some plans like `LocalRelation`, `LogicalRDD`, etc. override `sameResult` method, but not use canonicalized plan to compare so the `CacheManager` can't detect the plans are the same.

This pr modifies them to use canonicalized plan when override `sameResult` method.

## How was this patch tested?

Added a test to check if DataFrame with plan overriding sameResult but not using canonicalized plan to compare can cacheTable.

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

Closes #13638 from ueshin/issues/SPARK-15915.
2016-06-14 10:52:13 -07:00
gatorsmile bc02d01129 [SPARK-15655][SQL] Fix Wrong Partition Column Order when Fetching Partitioned Tables
#### What changes were proposed in this pull request?
When fetching the partitioned table, the output contains wrong results. The order of partition key values do not match the order of partition key columns in output schema. For example,

```SQL
CREATE TABLE table_with_partition(c1 string) PARTITIONED BY (p1 string,p2 string,p3 string,p4 string,p5 string)

INSERT OVERWRITE TABLE table_with_partition PARTITION (p1='a',p2='b',p3='c',p4='d',p5='e') SELECT 'blarr'

SELECT p1, p2, p3, p4, p5, c1 FROM table_with_partition
```
```
+---+---+---+---+---+-----+
| p1| p2| p3| p4| p5|   c1|
+---+---+---+---+---+-----+
|  d|  e|  c|  b|  a|blarr|
+---+---+---+---+---+-----+
```

The expected result should be
```
+---+---+---+---+---+-----+
| p1| p2| p3| p4| p5|   c1|
+---+---+---+---+---+-----+
|  a|  b|  c|  d|  e|blarr|
+---+---+---+---+---+-----+
```
This PR is to fix this by enforcing the order matches the table partition definition.

#### How was this patch tested?
Added a test case into `SQLQuerySuite`

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13400 from gatorsmile/partitionedTableFetch.
2016-06-14 09:58:06 -07:00
Sean Owen 6151d2641f [MINOR] Clean up several build warnings, mostly due to internal use of old accumulators
## What changes were proposed in this pull request?

Another PR to clean up recent build warnings. This particularly cleans up several instances of the old accumulator API usage in tests that are straightforward to update. I think this qualifies as "minor".

## How was this patch tested?

Jenkins

Author: Sean Owen <sowen@cloudera.com>

Closes #13642 from srowen/BuildWarnings.
2016-06-14 09:40:07 -07:00
Sean Zhong 6e8cdef0cf [SPARK-15914][SQL] Add deprecated method back to SQLContext for backward source code compatibility
## What changes were proposed in this pull request?

Revert partial changes in SPARK-12600, and add some deprecated method back to SQLContext for backward source code compatibility.

## How was this patch tested?

Manual test.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #13637 from clockfly/SPARK-15914.
2016-06-14 09:10:27 -07:00
Jeff Zhang 53bb030847 doc fix of HiveThriftServer
## What changes were proposed in this pull request?

Just minor doc fix.

\cc yhuai

Author: Jeff Zhang <zjffdu@apache.org>

Closes #13659 from zjffdu/doc_fix.
2016-06-14 14:28:40 +01:00
Wenchen Fan 688b6ef9dc [SPARK-15932][SQL][DOC] document the contract of encoder serializer expressions
## What changes were proposed in this pull request?

In our encoder framework, we imply that serializer expressions should use `BoundReference` to refer to the input object, and a lot of codes depend on this contract(e.g. ExpressionEncoder.tuple).  This PR adds some document and assert in `ExpressionEncoder` to make it clearer.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #13648 from cloud-fan/comment.
2016-06-13 22:02:23 -07:00
Sandeep Singh 1842cdd4ee [SPARK-15663][SQL] SparkSession.catalog.listFunctions shouldn't include the list of built-in functions
## What changes were proposed in this pull request?
SparkSession.catalog.listFunctions currently returns all functions, including the list of built-in functions. This makes the method not as useful because anytime it is run the result set contains over 100 built-in functions.

## How was this patch tested?
CatalogSuite

Author: Sandeep Singh <sandeep@techaddict.me>

Closes #13413 from techaddict/SPARK-15663.
2016-06-13 21:58:52 -07:00
gatorsmile 5827b65e28 [SPARK-15808][SQL] File Format Checking When Appending Data
#### What changes were proposed in this pull request?
**Issue:** Got wrong results or strange errors when append data to a table with mismatched file format.

_Example 1: PARQUET -> CSV_
```Scala
createDF(0, 9).write.format("parquet").saveAsTable("appendParquetToOrc")
createDF(10, 19).write.mode(SaveMode.Append).format("orc").saveAsTable("appendParquetToOrc")
```

Error we got:
```
Job aborted due to stage failure: Task 0 in stage 2.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2.0 (TID 2, localhost): java.lang.RuntimeException: file:/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/warehouse-bc8fedf2-aa6a-4002-a18b-524c6ac859d4/appendorctoparquet/part-r-00000-c0e3f365-1d46-4df5-a82c-b47d7af9feb9.snappy.orc is not a Parquet file. expected magic number at tail [80, 65, 82, 49] but found [79, 82, 67, 23]
```

_Example 2: Json -> CSV_
```Scala
createDF(0, 9).write.format("json").saveAsTable("appendJsonToCSV")
createDF(10, 19).write.mode(SaveMode.Append).format("parquet").saveAsTable("appendJsonToCSV")
```

No exception, but wrong results:
```
+----+----+
|  c1|  c2|
+----+----+
|null|null|
|null|null|
|null|null|
|null|null|
|   0|str0|
|   1|str1|
|   2|str2|
|   3|str3|
|   4|str4|
|   5|str5|
|   6|str6|
|   7|str7|
|   8|str8|
|   9|str9|
+----+----+
```
_Example 3: Json -> Text_
```Scala
createDF(0, 9).write.format("json").saveAsTable("appendJsonToText")
createDF(10, 19).write.mode(SaveMode.Append).format("text").saveAsTable("appendJsonToText")
```

Error we got:
```
Text data source supports only a single column, and you have 2 columns.
```

This PR is to issue an exception with appropriate error messages.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13546 from gatorsmile/fileFormatCheck.
2016-06-13 19:31:40 -07:00
Sean Zhong 7b9071eeaa [SPARK-15910][SQL] Check schema consistency when using Kryo encoder to convert DataFrame to Dataset
## What changes were proposed in this pull request?

This PR enforces schema check when converting DataFrame to Dataset using Kryo encoder. For example.

**Before the change:**

Schema is NOT checked when converting DataFrame to Dataset using kryo encoder.
```
scala> case class B(b: Int)
scala> implicit val encoder = Encoders.kryo[B]
scala> val df = Seq((1)).toDF("b")
scala> val ds = df.as[B] // Schema compatibility is NOT checked
```

**After the change:**
Report AnalysisException since the schema is NOT compatible.
```
scala> val ds = Seq((1)).toDF("b").as[B]
org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(`b` AS BINARY)' due to data type mismatch: cannot cast IntegerType to BinaryType;
...
```

## How was this patch tested?

Unit test.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #13632 from clockfly/spark-15910.
2016-06-13 17:43:55 -07:00
Josh Rosen a6babca1bf [SPARK-15929] Fix portability of DataFrameSuite path globbing tests
The DataFrameSuite regression tests for SPARK-13774 fail in my environment because they attempt to glob over all of `/mnt` and some of the subdirectories restrictive permissions which cause the test to fail.

This patch rewrites those tests to remove all environment-specific assumptions; the tests now create their own unique temporary paths for use in the tests.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #13649 from JoshRosen/SPARK-15929.
2016-06-13 17:06:22 -07:00
Wenchen Fan c4b1ad0209 [SPARK-15887][SQL] Bring back the hive-site.xml support for Spark 2.0
## What changes were proposed in this pull request?

Right now, Spark 2.0 does not load hive-site.xml. Based on users' feedback, it seems make sense to still load this conf file.

This PR adds a `hadoopConf` API in `SharedState`, which is `sparkContext.hadoopConfiguration` by default. When users are under hive context, `SharedState.hadoopConf` will load hive-site.xml and append its configs to `sparkContext.hadoopConfiguration`.

When we need to read hadoop config in spark sql, we should call `SessionState.newHadoopConf`, which contains `sparkContext.hadoopConfiguration`, hive-site.xml and sql configs.

## How was this patch tested?

new test in `HiveDataFrameSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #13611 from cloud-fan/hive-site.
2016-06-13 14:57:35 -07:00
Tathagata Das c654ae2140 [SPARK-15889][SQL][STREAMING] Add a unique id to ContinuousQuery
## What changes were proposed in this pull request?

ContinuousQueries have names that are unique across all the active ones. However, when queries are rapidly restarted with same name, it causes races conditions with the listener. A listener event from a stopped query can arrive after the query has been restarted, leading to complexities in monitoring infrastructure.

Along with this change, I have also consolidated all the messy code paths to start queries with different sinks.

## How was this patch tested?
Added unit tests, and existing unit tests.

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

Closes #13613 from tdas/SPARK-15889.
2016-06-13 13:44:46 -07:00
Takeshi YAMAMURO 5ad4e32d46 [SPARK-15530][SQL] Set #parallelism for file listing in listLeafFilesInParallel
## What changes were proposed in this pull request?
This pr is to set the number of parallelism to prevent file listing in `listLeafFilesInParallel` from generating many tasks in case of large #defaultParallelism.

## How was this patch tested?
Manually checked

Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes #13444 from maropu/SPARK-15530.
2016-06-13 13:41:26 -07:00
gatorsmile 3b7fb84cf8 [SPARK-15676][SQL] Disallow Column Names as Partition Columns For Hive Tables
#### What changes were proposed in this pull request?
When creating a Hive Table (not data source tables), a common error users might make is to specify an existing column name as a partition column. Below is what Hive returns in this case:
```
hive> CREATE TABLE partitioned (id bigint, data string) PARTITIONED BY (data string, part string);
FAILED: SemanticException [Error 10035]: Column repeated in partitioning columns
```
Currently, the error we issued is very confusing:
```
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:For direct MetaStore DB connections, we don't support retries at the client level.);
```
This PR is to fix the above issue by capturing the usage error in `Parser`.

#### How was this patch tested?
Added a test case to `DDLCommandSuite`

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13415 from gatorsmile/partitionColumnsInTableSchema.
2016-06-13 13:22:46 -07:00
Tathagata Das a6a18a4573 [HOTFIX][MINOR][SQL] Revert " Standardize 'continuous queries' to 'streaming D…
This reverts commit d32e227787.
Broke build - https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Compile/job/spark-branch-2.0-compile-maven-hadoop-2.3/326/console

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

Closes #13645 from tdas/build-break.
2016-06-13 12:47:47 -07:00
Liwei Lin d32e227787 [MINOR][SQL] Standardize 'continuous queries' to 'streaming Datasets/DataFrames'
## What changes were proposed in this pull request?

This patch does some replacing (as `streaming Datasets/DataFrames` is the term we've chosen in [SPARK-15593](00c310133d)):
 - `continuous queries` -> `streaming Datasets/DataFrames`
 - `non-continuous queries` -> `non-streaming Datasets/DataFrames`

This patch also adds `test("check foreach() can only be called on streaming Datasets/DataFrames")`.

## How was this patch tested?

N/A

Author: Liwei Lin <lwlin7@gmail.com>

Closes #13595 from lw-lin/continuous-queries-to-streaming-dss-dfs.
2016-06-13 11:49:15 -07:00
Wenchen Fan cd47e23374 [SPARK-15814][SQL] Aggregator can return null result
## What changes were proposed in this pull request?

It's similar to the bug fixed in https://github.com/apache/spark/pull/13425, we should consider null object and wrap the `CreateStruct` with `If` to do null check.

This PR also improves the test framework to test the objects of `Dataset[T]` directly, instead of calling `toDF` and compare the rows.

## How was this patch tested?

new test in `DatasetAggregatorSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #13553 from cloud-fan/agg-null.
2016-06-13 09:58:48 -07:00
Wenchen Fan e2ab79d5ea [SPARK-15898][SQL] DataFrameReader.text should return DataFrame
## What changes were proposed in this pull request?

We want to maintain API compatibility for DataFrameReader.text, and will introduce a new API called DataFrameReader.textFile which returns Dataset[String].

affected PRs:
https://github.com/apache/spark/pull/11731
https://github.com/apache/spark/pull/13104
https://github.com/apache/spark/pull/13184

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #13604 from cloud-fan/revert.
2016-06-12 21:36:41 -07:00
Herman van Hövell tot Westerflier 1f8f2b5c2a [SPARK-15370][SQL] Fix count bug
# What changes were proposed in this pull request?
This pull request fixes the COUNT bug in the `RewriteCorrelatedScalarSubquery` rule.

After this change, the rule tests the expression at the root of the correlated subquery to determine whether the expression returns `NULL` on empty input. If the expression does not return `NULL`, the rule generates additional logic in the `Project` operator above the rewritten subquery. This additional logic intercepts `NULL` values coming from the outer join and replaces them with the value that the subquery's expression would return on empty input.

This PR takes over https://github.com/apache/spark/pull/13155. It only fixes an issue with `Literal` construction and style issues.  All credits should go frreiss.

# How was this patch tested?
Added regression tests to cover all branches of the updated rule (see changes to `SubquerySuite`).
Ran all existing automated regression tests after merging with latest trunk.

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

Closes #13629 from hvanhovell/SPARK-15370-cleanup.
2016-06-12 21:30:32 -07:00
Wenchen Fan f5d38c3925 Revert "[SPARK-15753][SQL] Move Analyzer stuff to Analyzer from DataFrameWriter"
This reverts commit 0ec279ffdf.
2016-06-12 16:52:15 -07:00
Takuya UESHIN caebd7f262 [SPARK-15870][SQL] DataFrame can't execute after uncacheTable.
## What changes were proposed in this pull request?

If a cached `DataFrame` executed more than once and then do `uncacheTable` like the following:

```
    val selectStar = sql("SELECT * FROM testData WHERE key = 1")
    selectStar.createOrReplaceTempView("selectStar")

    spark.catalog.cacheTable("selectStar")
    checkAnswer(
      selectStar,
      Seq(Row(1, "1")))

    spark.catalog.uncacheTable("selectStar")
    checkAnswer(
      selectStar,
      Seq(Row(1, "1")))
```

, then the uncached `DataFrame` can't execute because of `Task not serializable` exception like:

```
org.apache.spark.SparkException: Task not serializable
	at org.apache.spark.util.ClosureCleaner$.ensureSerializable(ClosureCleaner.scala:298)
	at org.apache.spark.util.ClosureCleaner$.org$apache$spark$util$ClosureCleaner$$clean(ClosureCleaner.scala:288)
	at org.apache.spark.util.ClosureCleaner$.clean(ClosureCleaner.scala:108)
	at org.apache.spark.SparkContext.clean(SparkContext.scala:2038)
	at org.apache.spark.SparkContext.runJob(SparkContext.scala:1897)
	at org.apache.spark.SparkContext.runJob(SparkContext.scala:1912)
	at org.apache.spark.rdd.RDD$$anonfun$collect$1.apply(RDD.scala:884)
	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
	at org.apache.spark.rdd.RDD.withScope(RDD.scala:357)
	at org.apache.spark.rdd.RDD.collect(RDD.scala:883)
	at org.apache.spark.sql.execution.SparkPlan.executeCollect(SparkPlan.scala:290)
...
Caused by: java.lang.UnsupportedOperationException: Accumulator must be registered before send to executor
	at org.apache.spark.util.AccumulatorV2.writeReplace(AccumulatorV2.scala:153)
	at sun.reflect.GeneratedMethodAccessor2.invoke(Unknown Source)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at java.io.ObjectStreamClass.invokeWriteReplace(ObjectStreamClass.java:1118)
	at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1136)
	at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1548)
	at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1509)
	at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1432)
...
```

Notice that `DataFrame` uncached with `DataFrame.unpersist()` works, but with `spark.catalog.uncacheTable` doesn't work.

This pr reverts a part of cf38fe0 not to unregister `batchStats` accumulator, which is not needed to be unregistered here because it will be done by `ContextCleaner` after it is collected by GC.

## How was this patch tested?

Added a test to check if DataFrame can execute after uncacheTable and other existing tests.
But I made a test to check if the accumulator was cleared as `ignore` because the test would be flaky.

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

Closes #13596 from ueshin/issues/SPARK-15870.
2016-06-12 16:37:44 -07:00
Herman van Hovell 20b8f2c32a [SPARK-15370][SQL] Revert PR "Update RewriteCorrelatedSuquery rule"
This reverts commit 9770f6ee60.

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

Closes #13626 from hvanhovell/SPARK-15370-revert.
2016-06-12 15:06:37 -07:00
Ioana Delaney 0ff8a68b9f [SPARK-15832][SQL] Embedded IN/EXISTS predicate subquery throws TreeNodeException
## What changes were proposed in this pull request?
Queries with embedded existential sub-query predicates throws exception when building the physical plan.

Example failing query:
```SQL
scala> Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t1")
scala> Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t2")
scala> sql("select c1 from t1 where (case when c2 in (select c2 from t2) then 2 else 3 end) IN (select c2 from t1)").show()

Binding attribute, tree: c2#239
org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: c2#239
  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$.bindReference(BoundAttribute.scala:87)
  at org.apache.spark.sql.execution.joins.HashJoin$$anonfun$4.apply(HashJoin.scala:66)
  at org.apache.spark.sql.execution.joins.HashJoin$$anonfun$4.apply(HashJoin.scala:66)
  at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
  at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
  at scala.collection.immutable.List.foreach(List.scala:381)
  at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
  at scala.collection.immutable.List.map(List.scala:285)
  at org.apache.spark.sql.execution.joins.HashJoin$class.org$apache$spark$sql$execution$joins$HashJoin$$x$8(HashJoin.scala:66)
  at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.org$apache$spark$sql$execution$joins$HashJoin$$x$8$lzycompute(BroadcastHashJoinExec.scala:38)
  at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.org$apache$spark$sql$execution$joins$HashJoin$$x$8(BroadcastHashJoinExec.scala:38)
  at org.apache.spark.sql.execution.joins.HashJoin$class.buildKeys(HashJoin.scala:63)
  at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.buildKeys$lzycompute(BroadcastHashJoinExec.scala:38)
  at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.buildKeys(BroadcastHashJoinExec.scala:38)
  at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.requiredChildDistribution(BroadcastHashJoinExec.scala:52)
```

**Problem description:**
When the left hand side expression of an existential sub-query predicate contains another embedded sub-query predicate, the RewritePredicateSubquery optimizer rule does not resolve the embedded sub-query expressions into existential joins.For example, the above query has the following optimized plan, which fails during physical plan build.

```SQL
== Optimized Logical Plan ==
Project [_1#224 AS c1#227]
+- Join LeftSemi, (CASE WHEN predicate-subquery#255 [(_2#225 = c2#239)] THEN 2 ELSE 3 END = c2#228#262)
   :  +- SubqueryAlias predicate-subquery#255 [(_2#225 = c2#239)]
   :     +- LocalRelation [c2#239]
   :- LocalRelation [_1#224, _2#225]
   +- LocalRelation [c2#228#262]

== Physical Plan ==
org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: c2#239
```

**Solution:**
In RewritePredicateSubquery, before rewriting the outermost predicate sub-query, resolve any embedded existential sub-queries. The Optimized plan for the above query after the changes looks like below.

```SQL
== Optimized Logical Plan ==
Project [_1#224 AS c1#227]
+- Join LeftSemi, (CASE WHEN exists#285 THEN 2 ELSE 3 END = c2#228#284)
   :- Join ExistenceJoin(exists#285), (_2#225 = c2#239)
   :  :- LocalRelation [_1#224, _2#225]
   :  +- LocalRelation [c2#239]
   +- LocalRelation [c2#228#284]

== Physical Plan ==
*Project [_1#224 AS c1#227]
+- *BroadcastHashJoin [CASE WHEN exists#285 THEN 2 ELSE 3 END], [c2#228#284], LeftSemi, BuildRight
   :- *BroadcastHashJoin [_2#225], [c2#239], ExistenceJoin(exists#285), BuildRight
   :  :- LocalTableScan [_1#224, _2#225]
   :  +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)))
   :     +- LocalTableScan [c2#239]
   +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)))
      +- LocalTableScan [c2#228#284]
      +- LocalTableScan [c222#36], [[111],[222]]
```

## How was this patch tested?
Added new test cases in SubquerySuite.scala

Author: Ioana Delaney <ioanamdelaney@gmail.com>

Closes #13570 from ioana-delaney/fixEmbedSubPredV1.
2016-06-12 14:26:29 -07:00
frreiss 9770f6ee60 [SPARK-15370][SQL] Update RewriteCorrelatedScalarSubquery rule to fix COUNT bug
## What changes were proposed in this pull request?
This pull request fixes the COUNT bug in the `RewriteCorrelatedScalarSubquery` rule.

After this change, the rule tests the expression at the root of the correlated subquery to determine whether the expression returns NULL on empty input. If the expression does not return NULL, the rule generates additional logic in the Project operator above the rewritten subquery.  This additional logic intercepts NULL values coming from the outer join and replaces them with the value that the subquery's expression would return on empty input.

## How was this patch tested?
Added regression tests to cover all branches of the updated rule (see changes to `SubquerySuite.scala`).
Ran all existing automated regression tests after merging with latest trunk.

Author: frreiss <frreiss@us.ibm.com>

Closes #13155 from frreiss/master.
2016-06-12 14:21:10 -07:00
Sean Owen f51dfe616b [SPARK-15086][CORE][STREAMING] Deprecate old Java accumulator API
## What changes were proposed in this pull request?

- Deprecate old Java accumulator API; should use Scala now
- Update Java tests and examples
- Don't bother testing old accumulator API in Java 8 (too)
- (fix a misspelling too)

## How was this patch tested?

Jenkins tests

Author: Sean Owen <sowen@cloudera.com>

Closes #13606 from srowen/SPARK-15086.
2016-06-12 11:44:33 -07:00
hyukjinkwon 9e204c62c6 [SPARK-15840][SQL] Add two missing options in documentation and some option related changes
## What changes were proposed in this pull request?

This PR

1. Adds the documentations for some missing options, `inferSchema` and `mergeSchema` for Python and Scala.

2. Fiixes `[[DataFrame]]` to ```:class:`DataFrame` ``` so that this can be shown

  - from
    ![2016-06-09 9 31 16](https://cloud.githubusercontent.com/assets/6477701/15929721/8b864734-2e89-11e6-83f6-207527de4ac9.png)

  - to (with class link)
    ![2016-06-09 9 31 00](https://cloud.githubusercontent.com/assets/6477701/15929717/8a03d728-2e89-11e6-8a3f-08294964db22.png)

  (Please refer [the latest documentation](https://people.apache.org/~pwendell/spark-nightly/spark-master-docs/latest/api/python/pyspark.sql.html))

3. Moves `mergeSchema` option to `ParquetOptions` with removing unused options, `metastoreSchema` and `metastoreTableName`.

  They are not used anymore. They were removed in e720dda42e and there are no use cases as below:

  ```bash
  grep -r -e METASTORE_SCHEMA -e \"metastoreSchema\" -e \"metastoreTableName\" -e METASTORE_TABLE_NAME .
  ```

  ```
  ./sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala:  private[sql] val METASTORE_SCHEMA = "metastoreSchema"
  ./sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala:  private[sql] val METASTORE_TABLE_NAME = "metastoreTableName"
  ./sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala:        ParquetFileFormat.METASTORE_TABLE_NAME -> TableIdentifier(
```

  It only sets `metastoreTableName` in the last case but does not use the table name.

4. Sets the correct default values (in the documentation) for `compression` option for ORC(`snappy`, see [OrcOptions.scala#L33-L42](3ded5bc4db/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala (L33-L42))) and Parquet(`the value specified in SQLConf`, see [ParquetOptions.scala#L38-L47](3ded5bc4db/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala (L38-L47))) and `columnNameOfCorruptRecord` for JSON(`the value specified in SQLConf`, see [JsonFileFormat.scala#L53-L55](4538443e27/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala (L53-L55)) and [JsonFileFormat.scala#L105-L106](4538443e27/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala (L105-L106))).

## How was this patch tested?

Existing tests should cover this.

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

Closes #13576 from HyukjinKwon/SPARK-15840.
2016-06-11 23:20:40 -07:00
Eric Liang e1f986c7a3 [SPARK-15860] Metrics for codegen size and perf
## What changes were proposed in this pull request?

Adds codahale metrics for the codegen source text size and how long it takes to compile. The size is particularly interesting, since the JVM does have hard limits on how large methods can get.

To simplify, I added the metrics under a statically-initialized source that is always registered with SparkEnv.

## How was this patch tested?

Unit tests

Author: Eric Liang <ekl@databricks.com>

Closes #13586 from ericl/spark-15860.
2016-06-11 23:16:21 -07:00
Dongjoon Hyun 3fd2ff4dd8 [SPARK-15807][SQL] Support varargs for dropDuplicates in Dataset/DataFrame
## What changes were proposed in this pull request?
This PR adds `varargs`-types `dropDuplicates` functions in `Dataset/DataFrame`. Currently, `dropDuplicates` supports only `Seq` or `Array`.

**Before**
```scala
scala> val ds = spark.createDataFrame(Seq(("a", 1), ("b", 2), ("a", 2)))
ds: org.apache.spark.sql.DataFrame = [_1: string, _2: int]

scala> ds.dropDuplicates(Seq("_1", "_2"))
res0: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [_1: string, _2: int]

scala> ds.dropDuplicates("_1", "_2")
<console>:26: error: overloaded method value dropDuplicates with alternatives:
  (colNames: Array[String])org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] <and>
  (colNames: Seq[String])org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] <and>
  ()org.apache.spark.sql.Dataset[org.apache.spark.sql.Row]
 cannot be applied to (String, String)
       ds.dropDuplicates("_1", "_2")
          ^
```

**After**
```scala
scala> val ds = spark.createDataFrame(Seq(("a", 1), ("b", 2), ("a", 2)))
ds: org.apache.spark.sql.DataFrame = [_1: string, _2: int]

scala> ds.dropDuplicates("_1", "_2")
res0: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [_1: string, _2: int]
```

## How was this patch tested?

Pass the Jenkins tests with new testcases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13545 from dongjoon-hyun/SPARK-15807.
2016-06-11 15:47:51 -07:00