Commit graph

1749 commits

Author SHA1 Message Date
Herman van Hovell 2330f3ecbb [SPARK-16836][SQL] Add support for CURRENT_DATE/CURRENT_TIMESTAMP literals
## What changes were proposed in this pull request?
In Spark 1.6 (with Hive support) we could use `CURRENT_DATE` and `CURRENT_TIMESTAMP` functions as literals (without adding braces), for example:
```SQL
select /* Spark 1.6: */ current_date, /* Spark 1.6  & Spark 2.0: */ current_date()
```
This was accidentally dropped in Spark 2.0. This PR reinstates this functionality.

## How was this patch tested?
Added a case to ExpressionParserSuite.

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

Closes #14442 from hvanhovell/SPARK-16836.
2016-08-02 10:09:47 -07:00
Liang-Chi Hsieh 146001a9ff [SPARK-16062] [SPARK-15989] [SQL] Fix two bugs of Python-only UDTs
## What changes were proposed in this pull request?

There are two related bugs of Python-only UDTs. Because the test case of second one needs the first fix too. I put them into one PR. If it is not appropriate, please let me know.

### First bug: When MapObjects works on Python-only UDTs

`RowEncoder` will use `PythonUserDefinedType.sqlType` for its deserializer expression. If the sql type is `ArrayType`, we will have `MapObjects` working on it. But `MapObjects` doesn't consider `PythonUserDefinedType` as its input data type. It causes error like:

    import pyspark.sql.group
    from pyspark.sql.tests import PythonOnlyPoint, PythonOnlyUDT
    from pyspark.sql.types import *

    schema = StructType().add("key", LongType()).add("val", PythonOnlyUDT())
    df = spark.createDataFrame([(i % 3, PythonOnlyPoint(float(i), float(i))) for i in range(10)], schema=schema)
    df.show()

    File "/home/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 o36.showString.
    : java.lang.RuntimeException: Error while decoding: scala.MatchError: org.apache.spark.sql.types.PythonUserDefinedTypef4ceede8 (of class org.apache.spark.sql.types.PythonUserDefinedType)
    ...

### Second bug: When Python-only UDTs is the element type of ArrayType

    import pyspark.sql.group
    from pyspark.sql.tests import PythonOnlyPoint, PythonOnlyUDT
    from pyspark.sql.types import *

    schema = StructType().add("key", LongType()).add("val", ArrayType(PythonOnlyUDT()))
    df = spark.createDataFrame([(i % 3, [PythonOnlyPoint(float(i), float(i))]) for i in range(10)], schema=schema)
    df.show()

## How was this patch tested?
PySpark's sql tests.

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

Closes #13778 from viirya/fix-pyudt.
2016-08-02 10:08:18 -07:00
Tom Magrino 1dab63d8d3 [SPARK-16837][SQL] TimeWindow incorrectly drops slideDuration in constructors
## What changes were proposed in this pull request?

Fix of incorrect arguments (dropping slideDuration and using windowDuration) in constructors for TimeWindow.

The JIRA this addresses is here: https://issues.apache.org/jira/browse/SPARK-16837

## How was this patch tested?

Added a test to TimeWindowSuite to check that the results of TimeWindow object apply and TimeWindow class constructor are equivalent.

Author: Tom Magrino <tmagrino@fb.com>

Closes #14441 from tmagrino/windowing-fix.
2016-08-02 09:16:44 -07:00
petermaxlee a1ff72e1cc [SPARK-16850][SQL] Improve type checking error message for greatest/least
## What changes were proposed in this pull request?
Greatest/least function does not have the most friendly error message for data types. This patch improves the error message to not show the Seq type, and use more human readable data types.

Before:
```
org.apache.spark.sql.AnalysisException: cannot resolve 'greatest(CAST(1.0 AS DECIMAL(2,1)), "1.0")' due to data type mismatch: The expressions should all have the same type, got GREATEST (ArrayBuffer(DecimalType(2,1), StringType)).; line 1 pos 7
```

After:
```
org.apache.spark.sql.AnalysisException: cannot resolve 'greatest(CAST(1.0 AS DECIMAL(2,1)), "1.0")' due to data type mismatch: The expressions should all have the same type, got GREATEST(decimal(2,1), string).; line 1 pos 7
```

## How was this patch tested?
Manually verified the output and also added unit tests to ConditionalExpressionSuite.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14453 from petermaxlee/SPARK-16850.
2016-08-02 19:32:35 +08:00
Wenchen Fan 2eedc00b04 [SPARK-16828][SQL] remove MaxOf and MinOf
## What changes were proposed in this pull request?

These 2 expressions are not needed anymore after we have `Greatest` and `Least`. This PR removes them and related tests.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14434 from cloud-fan/minor1.
2016-08-01 17:54:41 -07:00
Holden Karau ab1e761f96 [SPARK-16774][SQL] Fix use of deprecated timestamp constructor & improve timezone handling
## What changes were proposed in this pull request?

Removes the deprecated timestamp constructor and incidentally fixes the use which was using system timezone rather than the one specified when working near DST.

This change also causes the roundtrip tests to fail since it now actually uses all the timezones near DST boundaries where it didn't before.

Note: this is only a partial the solution, longer term we should follow up with https://issues.apache.org/jira/browse/SPARK-16788 to avoid this problem & simplify our timezone handling code.

## How was this patch tested?

New tests for two timezones added so even if user timezone happens to coincided with one, the other tests should still fail. Important note: this (temporarily) disables the round trip tests until we can fix the issue more thoroughly.

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

Closes #14398 from holdenk/SPARK-16774-fix-use-of-deprecated-timestamp-constructor.
2016-08-01 13:57:05 -07:00
eyal farago 338a98d65c [SPARK-16791][SQL] cast struct with timestamp field fails
## What changes were proposed in this pull request?
a failing test case + fix to SPARK-16791 (https://issues.apache.org/jira/browse/SPARK-16791)

## How was this patch tested?
added a failing test case to CastSuit, then fixed the Cast code and rerun the entire CastSuit

Author: eyal farago <eyal farago>
Author: Eyal Farago <eyal.farago@actimize.com>

Closes #14400 from eyalfa/SPARK-16791_cast_struct_with_timestamp_field_fails.
2016-08-01 22:43:32 +08:00
Dongjoon Hyun 64d8f37c71 [SPARK-16726][SQL] Improve Union/Intersect/Except error messages on incompatible types
## What changes were proposed in this pull request?

Currently, `UNION` queries on incompatible types show misleading error messages, i.e., `unresolved operator Union`. We had better show a more correct message. This will help users in the situation of [SPARK-16704](https://issues.apache.org/jira/browse/SPARK-16704).

**Before**
```scala
scala> sql("select 1,2,3 union (select 1,array(2),3)")
org.apache.spark.sql.AnalysisException: unresolved operator 'Union;
scala> sql("select 1,2,3 intersect (select 1,array(2),3)")
org.apache.spark.sql.AnalysisException: unresolved operator 'Intersect;
scala> sql("select 1,2,3 except (select 1,array(2),3)")
org.apache.spark.sql.AnalysisException: unresolved operator 'Except;
```

**After**
```scala
scala> sql("select 1,2,3 union (select 1,array(2),3)")
org.apache.spark.sql.AnalysisException: Union can only be performed on tables with the compatible column types. ArrayType(IntegerType,false) <> IntegerType at the second column of the second table;
scala> sql("select 1,2,3 intersect (select 1,array(2),3)")
org.apache.spark.sql.AnalysisException: Intersect can only be performed on tables with the compatible column types. ArrayType(IntegerType,false) <> IntegerType at the second column of the second table;
scala> sql("select 1,2,3 except (select array(1),array(2),3)")
org.apache.spark.sql.AnalysisException: Except can only be performed on tables with the compatible column types. ArrayType(IntegerType,false) <> IntegerType at the first column of the second table;
```

## How was this patch tested?

Pass the Jenkins test with a new test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14355 from dongjoon-hyun/SPARK-16726.
2016-08-01 11:12:58 +02:00
Wenchen Fan 301fb0d723 [SPARK-16731][SQL] use StructType in CatalogTable and remove CatalogColumn
## What changes were proposed in this pull request?

`StructField` has very similar semantic with `CatalogColumn`, except that `CatalogColumn` use string to express data type. I think it's reasonable to use `StructType` as the `CatalogTable.schema` and remove `CatalogColumn`.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14363 from cloud-fan/column.
2016-07-31 18:18:53 -07:00
Reynold Xin 064d91ff73 [SPARK-16813][SQL] Remove private[sql] and private[spark] from catalyst package
## What changes were proposed in this pull request?
The catalyst package is meant to be internal, and as a result it does not make sense to mark things as private[sql] or private[spark]. It simply makes debugging harder when Spark developers need to inspect the plans at runtime.

This patch removes all private[sql] and private[spark] visibility modifiers in org.apache.spark.sql.catalyst.

## How was this patch tested?
N/A - just visibility changes.

Author: Reynold Xin <rxin@databricks.com>

Closes #14418 from rxin/SPARK-16813.
2016-07-31 16:31:06 +08:00
Tathagata Das bbc247548a [SPARK-16748][SQL] SparkExceptions during planning should not wrapped in TreeNodeException
## What changes were proposed in this pull request?
We do not want SparkExceptions from job failures in the planning phase to create TreeNodeException. Hence do not wrap SparkException in TreeNodeException.

## How was this patch tested?
New unit test

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

Closes #14395 from tdas/SPARK-16748.
2016-07-29 19:59:35 -07:00
Liang-Chi Hsieh 9ade77c3fa [SPARK-16639][SQL] The query with having condition that contains grouping by column should work
## What changes were proposed in this pull request?

The query with having condition that contains grouping by column will be failed during analysis. E.g.,

    create table tbl(a int, b string);
    select count(b) from tbl group by a + 1 having a + 1 = 2;

Having condition should be able to use grouping by column.

## How was this patch tested?

Jenkins tests.

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

Closes #14296 from viirya/having-contains-grouping-column.
2016-07-28 22:33:33 +08:00
petermaxlee 11d427c924 [SPARK-16730][SQL] Implement function aliases for type casts
## What changes were proposed in this pull request?
Spark 1.x supports using the Hive type name as function names for doing casts, e.g.
```sql
SELECT int(1.0);
SELECT string(2.0);
```

The above query would work in Spark 1.x because Spark 1.x fail back to Hive for unimplemented functions, and break in Spark 2.0 because the fall back was removed.

This patch implements function aliases using an analyzer rule for the following cast functions:
- boolean
- tinyint
- smallint
- int
- bigint
- float
- double
- decimal
- date
- timestamp
- binary
- string

## How was this patch tested?
Added end-to-end tests in SQLCompatibilityFunctionSuite.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14364 from petermaxlee/SPARK-16730-2.
2016-07-28 13:13:17 +08:00
petermaxlee ef0ccbcb07 [SPARK-16729][SQL] Throw analysis exception for invalid date casts
## What changes were proposed in this pull request?
Spark currently throws exceptions for invalid casts for all other data types except date type. Somehow date type returns null. It should be consistent and throws analysis exception as well.

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

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14358 from petermaxlee/SPARK-16729.
2016-07-27 16:04:43 +08:00
Qifan Pu 738b4cc548 [SPARK-16524][SQL] Add RowBatch and RowBasedHashMapGenerator
## What changes were proposed in this pull request?

This PR is the first step for the following feature:

For hash aggregation in Spark SQL, we use a fast aggregation hashmap to act as a "cache" in order to boost aggregation performance. Previously, the hashmap is backed by a `ColumnarBatch`. This has performance issues when we have wide schema for the aggregation table (large number of key fields or value fields).
In this JIRA, we support another implementation of fast hashmap, which is backed by a `RowBasedKeyValueBatch`. We then automatically pick between the two implementations based on certain knobs.

In this first-step PR, implementations for `RowBasedKeyValueBatch` and `RowBasedHashMapGenerator` are added.

## How was this patch tested?

Unit tests: `RowBasedKeyValueBatchSuite`

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

Closes #14349 from ooq/SPARK-16524.
2016-07-26 18:08:07 -07:00
Wenchen Fan 6959061f02 [SPARK-16706][SQL] support java map in encoder
## What changes were proposed in this pull request?

finish the TODO, create a new expression `ExternalMapToCatalyst` to iterate the map directly.

## How was this patch tested?

new test in `JavaDatasetSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14344 from cloud-fan/java-map.
2016-07-26 15:33:05 +08:00
Liang-Chi Hsieh 7b06a8948f [SPARK-16686][SQL] Remove PushProjectThroughSample since it is handled by ColumnPruning
## What changes were proposed in this pull request?

We push down `Project` through `Sample` in `Optimizer` by the rule `PushProjectThroughSample`. However, if the projected columns produce new output, they will encounter whole data instead of sampled data. It will bring some inconsistency between original plan (Sample then Project) and optimized plan (Project then Sample). In the extreme case such as attached in the JIRA, if the projected column is an UDF which is supposed to not see the sampled out data, the result of UDF will be incorrect.

Since the rule `ColumnPruning` already handles general `Project` pushdown. We don't need  `PushProjectThroughSample` anymore. The rule `ColumnPruning` also avoids the described issue.

## How was this patch tested?

Jenkins tests.

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

Closes #14327 from viirya/fix-sample-pushdown.
2016-07-26 12:00:01 +08:00
Yin Huai 815f3eece5 [SPARK-16633][SPARK-16642][SPARK-16721][SQL] Fixes three issues related to lead and lag functions
## What changes were proposed in this pull request?
This PR contains three changes.

First, this PR changes the behavior of lead/lag back to Spark 1.6's behavior, which is described as below:
1. lead/lag respect null input values, which means that if the offset row exists and the input value is null, the result will be null instead of the default value.
2. If the offset row does not exist, the default value will be used.
3. OffsetWindowFunction's nullable setting also considers the nullability of its input (because of the first change).

Second, this PR fixes the evaluation of lead/lag when the input expression is a literal. This fix is a result of the first change. In current master, if a literal is used as the input expression of a lead or lag function, the result will be this literal even if the offset row does not exist.

Third, this PR makes ResolveWindowFrame not fire if a window function is not resolved.

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

Author: Yin Huai <yhuai@databricks.com>

Closes #14284 from yhuai/lead-lag.
2016-07-25 20:58:07 -07:00
Michael Armbrust f99e34e8e5 [SPARK-16724] Expose DefinedByConstructorParams
We don't generally make things in catalyst/execution private.  Instead they are just undocumented due to their lack of stability guarantees.

Author: Michael Armbrust <michael@databricks.com>

Closes #14356 from marmbrus/patch-1.
2016-07-25 20:41:24 -07:00
gatorsmile 3fc4566941 [SPARK-16678][SPARK-16677][SQL] Fix two View-related bugs
## What changes were proposed in this pull request?
**Issue 1: Disallow Creating/Altering a View when the same-name Table Exists (without IF NOT EXISTS)**
When we create OR alter a view, we check whether the view already exists. In the current implementation, if a table with the same name exists, we treat it as a view. However, this is not the right behavior. We should follow what Hive does. For example,
```
hive> CREATE TABLE tab1 (id int);
OK
Time taken: 0.196 seconds
hive> CREATE OR REPLACE VIEW tab1 AS SELECT * FROM t1;
FAILED: SemanticException [Error 10218]: Existing table is not a view
 The following is an existing table, not a view: default.tab1
hive> ALTER VIEW tab1 AS SELECT * FROM t1;
FAILED: SemanticException [Error 10218]: Existing table is not a view
 The following is an existing table, not a view: default.tab1
hive> CREATE VIEW IF NOT EXISTS tab1 AS SELECT * FROM t1;
OK
Time taken: 0.678 seconds
```

**Issue 2: Strange Error when Issuing Load Table Against A View**
Users should not be allowed to issue LOAD DATA against a view. Currently, when users doing it, we got a very strange runtime error. For example,
```SQL
LOAD DATA LOCAL INPATH "$testData" INTO TABLE $viewName
```
```
java.lang.reflect.InvocationTargetException was thrown.
java.lang.reflect.InvocationTargetException
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:606)
	at org.apache.spark.sql.hive.client.Shim_v0_14.loadTable(HiveShim.scala:680)
```
## How was this patch tested?
Added test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14314 from gatorsmile/tableDDLAgainstView.
2016-07-26 09:32:29 +08:00
Cheng Lian 7ea6d282b9 [SPARK-16703][SQL] Remove extra whitespace in SQL generation for window functions
## What changes were proposed in this pull request?

This PR fixes a minor formatting issue of `WindowSpecDefinition.sql` when no partitioning expressions are present.

Before:

```sql
( ORDER BY `a` ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)
```

After:

```sql
(ORDER BY `a` ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)
```

## How was this patch tested?

New test case added in `ExpressionSQLBuilderSuite`.

Author: Cheng Lian <lian@databricks.com>

Closes #14334 from liancheng/window-spec-sql-format.
2016-07-25 09:42:39 -07:00
hyukjinkwon 79826f3c79 [SPARK-16698][SQL] Field names having dots should be allowed for datasources based on FileFormat
## What changes were proposed in this pull request?

It seems this is a regression assuming from https://issues.apache.org/jira/browse/SPARK-16698.

Field name having dots throws an exception. For example the codes below:

```scala
val path = "/tmp/path"
val json =""" {"a.b":"data"}"""
spark.sparkContext
  .parallelize(json :: Nil)
  .saveAsTextFile(path)
spark.read.json(path).collect()
```

throws an exception as below:

```
Unable to resolve a.b given [a.b];
org.apache.spark.sql.AnalysisException: Unable to resolve a.b given [a.b];
	at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1$$anonfun$apply$5.apply(LogicalPlan.scala:134)
	at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1$$anonfun$apply$5.apply(LogicalPlan.scala:134)
	at scala.Option.getOrElse(Option.scala:121)
```

This problem was introduced in 17eec0a71b (diff-27c76f96a7b2733ecfd6f46a1716e153R121)

When extracting the data columns, it does not count that it can contains dots in field names. Actually, it seems the fields name are not expected as quoted when defining schema. So, It not have to consider whether this is wrapped with quotes because the actual schema (inferred or user-given schema) would not have the quotes for fields.

For example, this throws an exception. (**Loading JSON from RDD is fine**)

```scala
val json =""" {"a.b":"data"}"""
val rdd = spark.sparkContext.parallelize(json :: Nil)
spark.read.schema(StructType(Seq(StructField("`a.b`", StringType, true))))
  .json(rdd).select("`a.b`").printSchema()
```

as below:

```
cannot resolve '```a.b```' given input columns: [`a.b`];
org.apache.spark.sql.AnalysisException: cannot resolve '```a.b```' given input columns: [`a.b`];
	at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
```

## How was this patch tested?

Unit tests in `FileSourceStrategySuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14339 from HyukjinKwon/SPARK-16698-regression.
2016-07-25 22:51:30 +08:00
Wenchen Fan 64529b186a [SPARK-16691][SQL] move BucketSpec to catalyst module and use it in CatalogTable
## What changes were proposed in this pull request?

It's weird that we have `BucketSpec` to abstract bucket info, but don't use it in `CatalogTable`. This PR moves `BucketSpec` into catalyst module.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14331 from cloud-fan/check.
2016-07-25 22:05:48 +08:00
Wenchen Fan d27d362eba [SPARK-16660][SQL] CreateViewCommand should not take CatalogTable
## What changes were proposed in this pull request?

`CreateViewCommand` only needs some information of a `CatalogTable`, but not all of them. We have some tricks(e.g. we need to check the table type is `VIEW`, we need to make `CatalogColumn.dataType` nullable) to allow it to take a `CatalogTable`.
This PR cleans it up and only pass in necessary information to `CreateViewCommand`.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14297 from cloud-fan/minor2.
2016-07-25 22:02:00 +08:00
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
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
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
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
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
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 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
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
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
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
蒋星博 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
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
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
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
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
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 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
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
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
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 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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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 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
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
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
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
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
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
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
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