Commit graph

1509 commits

Author SHA1 Message Date
Marco Gaido b3d8fc3dc4 [SPARK-22226][SQL] splitExpression can create too many method calls in the outer class
## What changes were proposed in this pull request?

SPARK-18016 introduced `NestedClass` to avoid that the many methods generated by `splitExpressions` contribute to the outer class' constant pool, making it growing too much. Unfortunately, despite their definition is stored in the `NestedClass`, they all are invoked in the outer class and for each method invocation, there are two entries added to the constant pool: a `Methodref` and a `Utf8` entry (you can easily check this compiling a simple sample class with `janinoc` and looking at its Constant Pool). This limits the scalability of the solution with very large methods which are split in a lot of small ones. This means that currently we are generating classes like this one:

```
class SpecificUnsafeProjection extends org.apache.spark.sql.catalyst.expressions.UnsafeProjection {
...
  public UnsafeRow apply(InternalRow i) {
     rowWriter.zeroOutNullBytes();
     apply_0(i);
     apply_1(i);
...
    nestedClassInstance.apply_862(i);
    nestedClassInstance.apply_863(i);
...
    nestedClassInstance1.apply_1612(i);
    nestedClassInstance1.apply_1613(i);
...
  }
...
  private class NestedClass {
    private void apply_862(InternalRow i) { ... }
    private void apply_863(InternalRow i) { ... }
...
  }
  private class NestedClass1 {
    private void apply_1612(InternalRow i) { ... }
    private void apply_1613(InternalRow i) { ... }
...
  }
}
```

This PR reduce the Constant Pool size of the outer class by adding a new method to each nested class: in this method we invoke all the small methods generated by `splitExpression` in that nested class. In this way, in the outer class there is only one method invocation per nested class, reducing by orders of magnitude the entries in its constant pool because of method invocations. This means that after the patch the generated code becomes:

```
class SpecificUnsafeProjection extends org.apache.spark.sql.catalyst.expressions.UnsafeProjection {
...
  public UnsafeRow apply(InternalRow i) {
     rowWriter.zeroOutNullBytes();
     apply_0(i);
     apply_1(i);
     ...
     nestedClassInstance.apply(i);
     nestedClassInstance1.apply(i);
     ...
  }
...
  private class NestedClass {
    private void apply_862(InternalRow i) { ... }
    private void apply_863(InternalRow i) { ... }
...
    private void apply(InternalRow i) {
      apply_862(i);
      apply_863(i);
      ...
    }
  }
  private class NestedClass1 {
    private void apply_1612(InternalRow i) { ... }
    private void apply_1613(InternalRow i) { ... }
...
    private void apply(InternalRow i) {
      apply_1612(i);
      apply_1613(i);
      ...
    }
  }
}
```

## How was this patch tested?

Added UT and existing UTs

Author: Marco Gaido <mgaido@hortonworks.com>
Author: Marco Gaido <marcogaido91@gmail.com>

Closes #19480 from mgaido91/SPARK-22226.
2017-10-27 13:43:09 -07:00
Nathan Kronenfeld 592cfeab9c [SPARK-22308] Support alternative unit testing styles in external applications
## What changes were proposed in this pull request?
Support unit tests of external code (i.e., applications that use spark) using scalatest that don't want to use FunSuite.  SharedSparkContext already supports this, but SharedSQLContext does not.

I've introduced SharedSparkSession as a parent to SharedSQLContext, written in a way that it does support all scalatest styles.

## How was this patch tested?
There are three new unit test suites added that just test using FunSpec, FlatSpec, and WordSpec.

Author: Nathan Kronenfeld <nicole.oresme@gmail.com>

Closes #19529 from nkronenfeld/alternative-style-tests-2.
2017-10-26 00:29:49 -07:00
Ruben Berenguel Montoro 427359f077 [SPARK-13947][SQL] The error message from using an invalid column reference is not clear
## What changes were proposed in this pull request?

 Rewritten error message for clarity. Added extra information in case of attribute name collision, hinting the user to double-check referencing two different tables

## How was this patch tested?

No functional changes, only final message has changed. It has been tested manually against the situation proposed in the JIRA ticket. Automated tests in repository pass.

This PR is original work from me and I license this work to the Spark project

Author: Ruben Berenguel Montoro <ruben@mostlymaths.net>
Author: Ruben Berenguel Montoro <ruben@dreamattic.com>
Author: Ruben Berenguel <ruben@mostlymaths.net>

Closes #17100 from rberenguel/SPARK-13947-error-message.
2017-10-24 23:02:11 -07:00
Marco Gaido 3f5ba968c5 [SPARK-22301][SQL] Add rule to Optimizer for In with not-nullable value and empty list
## What changes were proposed in this pull request?

For performance reason, we should resolve in operation on an empty list as false in the optimizations phase, ad discussed in #19522.

## How was this patch tested?
Added UT

cc gatorsmile

Author: Marco Gaido <marcogaido91@gmail.com>
Author: Marco Gaido <mgaido@hortonworks.com>

Closes #19523 from mgaido91/SPARK-22301.
2017-10-24 09:11:52 -07:00
Zhenhua Wang f6290aea24 [SPARK-22285][SQL] Change implementation of ApproxCountDistinctForIntervals to TypedImperativeAggregate
## What changes were proposed in this pull request?

The current implementation of `ApproxCountDistinctForIntervals` is `ImperativeAggregate`. The number of `aggBufferAttributes` is the number of total words in the hllppHelper array. Each hllppHelper has 52 words by default relativeSD.

Since this aggregate function is used in equi-height histogram generation, and the number of buckets in histogram is usually hundreds, the number of `aggBufferAttributes` can easily reach tens of thousands or even more.

This leads to a huge method in codegen and causes error:
```
org.codehaus.janino.JaninoRuntimeException: Code of method "apply(Lorg/apache/spark/sql/catalyst/InternalRow;)Lorg/apache/spark/sql/catalyst/expressions/UnsafeRow;" of class "org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection" grows beyond 64 KB.
```
Besides, huge generated methods also result in performance regression.

In this PR, we change its implementation to `TypedImperativeAggregate`. After the fix, `ApproxCountDistinctForIntervals` can deal with more than thousands endpoints without throwing codegen error, and improve performance from `20 sec` to `2 sec` in a test case of 500 endpoints.

## How was this patch tested?

Test by an added test case and existing tests.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #19506 from wzhfy/change_forIntervals_typedAgg.
2017-10-23 23:02:36 +01:00
Dongjoon Hyun 6412ea1759 [SPARK-21247][SQL] Type comparison should respect case-sensitive SQL conf
## What changes were proposed in this pull request?

This is an effort to reduce the difference between Hive and Spark. Spark supports case-sensitivity in columns. Especially, for Struct types, with `spark.sql.caseSensitive=true`, the following is supported.

```scala
scala> sql("select named_struct('a', 1, 'A', 2).a").show
+--------------------------+
|named_struct(a, 1, A, 2).a|
+--------------------------+
|                         1|
+--------------------------+

scala> sql("select named_struct('a', 1, 'A', 2).A").show
+--------------------------+
|named_struct(a, 1, A, 2).A|
+--------------------------+
|                         2|
+--------------------------+
```

And vice versa, with `spark.sql.caseSensitive=false`, the following is supported.
```scala
scala> sql("select named_struct('a', 1).A, named_struct('A', 1).a").show
+--------------------+--------------------+
|named_struct(a, 1).A|named_struct(A, 1).a|
+--------------------+--------------------+
|                   1|                   1|
+--------------------+--------------------+
```

However, types are considered different. For example, SET operations fail.
```scala
scala> sql("SELECT named_struct('a',1) union all (select named_struct('A',2))").show
org.apache.spark.sql.AnalysisException: Union can only be performed on tables with the compatible column types. struct<A:int> <> struct<a:int> at the first column of the second table;;
'Union
:- Project [named_struct(a, 1) AS named_struct(a, 1)#57]
:  +- OneRowRelation$
+- Project [named_struct(A, 2) AS named_struct(A, 2)#58]
   +- OneRowRelation$
```

This PR aims to support case-insensitive type equality. For example, in Set operation, the above operation succeed when `spark.sql.caseSensitive=false`.

```scala
scala> sql("SELECT named_struct('a',1) union all (select named_struct('A',2))").show
+------------------+
|named_struct(a, 1)|
+------------------+
|               [1]|
|               [2]|
+------------------+
```

## How was this patch tested?

Pass the Jenkins with a newly add test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #18460 from dongjoon-hyun/SPARK-21247.
2017-10-14 00:35:12 +08:00
Wang Gengliang 2f00a71a87 [SPARK-22257][SQL] Reserve all non-deterministic expressions in ExpressionSet
## What changes were proposed in this pull request?

For non-deterministic expressions, they should be considered as not contained in the [[ExpressionSet]].
This is consistent with how we define `semanticEquals` between two expressions.
Otherwise, combining expressions will remove non-deterministic expressions which should be reserved.
E.g.
Combine filters of
```scala
testRelation.where(Rand(0) > 0.1).where(Rand(0) > 0.1)
```
should result in
```scala
testRelation.where(Rand(0) > 0.1 && Rand(0) > 0.1)
```

## How was this patch tested?

Unit test

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #19475 from gengliangwang/non-deterministic-expressionSet.
2017-10-12 22:45:19 -07:00
Zhenhua Wang 655f6f86f8 [SPARK-22208][SQL] Improve percentile_approx by not rounding up targetError and starting from index 0
## What changes were proposed in this pull request?

Currently percentile_approx never returns the first element when percentile is in (relativeError, 1/N], where relativeError default 1/10000, and N is the total number of elements. But ideally, percentiles in [0, 1/N] should all return the first element as the answer.

For example, given input data 1 to 10, if a user queries 10% (or even less) percentile, it should return 1, because the first value 1 already reaches 10%. Currently it returns 2.

Based on the paper, targetError is not rounded up, and searching index should start from 0 instead of 1. By following the paper, we should be able to fix the cases mentioned above.

## How was this patch tested?

Added a new test case and fix existing test cases.

Author: Zhenhua Wang <wzh_zju@163.com>

Closes #19438 from wzhfy/improve_percentile_approx.
2017-10-11 00:16:12 -07:00
Kazuaki Ishizaki 76fb173dd6 [SPARK-21751][SQL] CodeGeneraor.splitExpressions counts code size more precisely
## What changes were proposed in this pull request?

Current `CodeGeneraor.splitExpressions` splits statements into methods if the total length of statements is more than 1024 characters. The length may include comments or empty line.

This PR excludes comment or empty line from the length to reduce the number of generated methods in a class, by using `CodeFormatter.stripExtraNewLinesAndComments()` method.

## How was this patch tested?

Existing tests

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

Closes #18966 from kiszk/SPARK-21751.
2017-10-10 20:29:02 -07:00
gatorsmile 633ffd816d rename the file. 2017-10-10 11:01:02 -07:00
Feng Liu bebd2e1ce1 [SPARK-22222][CORE] Fix the ARRAY_MAX in BufferHolder and add a test
## What changes were proposed in this pull request?

We should not break the assumption that the length of the allocated byte array is word rounded:
https://github.com/apache/spark/blob/master/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java#L170
So we want to use `Integer.MAX_VALUE - 15` instead of `Integer.MAX_VALUE - 8` as the upper bound of an allocated byte array.

cc: srowen gatorsmile
## How was this patch tested?

Since the Spark unit test JVM has less than 1GB heap, here we run the test code as a submit job, so it can run on a JVM has 4GB memory.

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

Author: Feng Liu <fengliu@databricks.com>

Closes #19460 from liufengdb/fix_array_max.
2017-10-09 21:34:37 -07:00
Liang-Chi Hsieh debcbec749 [SPARK-21947][SS] Check and report error when monotonically_increasing_id is used in streaming query
## What changes were proposed in this pull request?

`monotonically_increasing_id` doesn't work in Structured Streaming. We should throw an exception if a streaming query uses it.

## How was this patch tested?

Added test.

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

Closes #19336 from viirya/SPARK-21947.
2017-10-06 13:10:04 -07:00
Wenchen Fan bb035f1ee5 [SPARK-22169][SQL] support byte length literal as identifier
## What changes were proposed in this pull request?

By definition the table name in Spark can be something like `123x`, `25a`, etc., with exceptions for literals like `12L`, `23BD`, etc. However, Spark SQL has a special byte length literal, which stops users to use digits followed by `b`, `k`, `m`, `g` as identifiers.

byte length literal is not a standard sql literal and is only used in the `tableSample` parser rule. This PR move the parsing of byte length literal from lexer to parser, so that users can use it as identifiers.

## How was this patch tested?

regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19392 from cloud-fan/parser-bug.
2017-10-04 13:13:51 -07:00
Takeshi Yamamuro 4a779bdac3 [SPARK-21871][SQL] Check actual bytecode size when compiling generated code
## What changes were proposed in this pull request?
This pr added code to check actual bytecode size when compiling generated code. In #18810, we added code to give up code compilation and use interpreter execution in `SparkPlan` if the line number of generated functions goes over `maxLinesPerFunction`. But, we already have code to collect metrics for compiled bytecode size in `CodeGenerator` object. So,we could easily reuse the code for this purpose.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #19083 from maropu/SPARK-21871.
2017-10-04 10:08:24 -07:00
Jose Torres 3099c574c5 [SPARK-22136][SS] Implement stream-stream outer joins.
## What changes were proposed in this pull request?

Allow one-sided outer joins between two streams when a watermark is defined.

## How was this patch tested?

new unit tests

Author: Jose Torres <jose@databricks.com>

Closes #19327 from joseph-torres/outerjoin.
2017-10-03 21:42:51 -07:00
gatorsmile 5f69433453 [SPARK-22171][SQL] Describe Table Extended Failed when Table Owner is Empty
## What changes were proposed in this pull request?

Users could hit `java.lang.NullPointerException` when the tables were created by Hive and the table's owner is `null` that are got from Hive metastore. `DESC EXTENDED` failed with the error:

> SQLExecutionException: java.lang.NullPointerException at scala.collection.immutable.StringOps$.length$extension(StringOps.scala:47) at scala.collection.immutable.StringOps.length(StringOps.scala:47) at scala.collection.IndexedSeqOptimized$class.isEmpty(IndexedSeqOptimized.scala:27) at scala.collection.immutable.StringOps.isEmpty(StringOps.scala:29) at scala.collection.TraversableOnce$class.nonEmpty(TraversableOnce.scala:111) at scala.collection.immutable.StringOps.nonEmpty(StringOps.scala:29) at org.apache.spark.sql.catalyst.catalog.CatalogTable.toLinkedHashMap(interface.scala:300) at org.apache.spark.sql.execution.command.DescribeTableCommand.describeFormattedTableInfo(tables.scala:565) at org.apache.spark.sql.execution.command.DescribeTableCommand.run(tables.scala:543) at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:66) at

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19395 from gatorsmile/desc.
2017-10-03 21:27:58 -07:00
Zhenhua Wang 365a29bdbf [SPARK-22100][SQL] Make percentile_approx support date/timestamp type and change the output type to be the same as input type
## What changes were proposed in this pull request?

The `percentile_approx` function previously accepted numeric type input and output double type results.

But since all numeric types, date and timestamp types are represented as numerics internally, `percentile_approx` can support them easily.

After this PR, it supports date type, timestamp type and numeric types as input types. The result type is also changed to be the same as the input type, which is more reasonable for percentiles.

This change is also required when we generate equi-height histograms for these types.

## How was this patch tested?

Added a new test and modified some existing tests.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #19321 from wzhfy/approx_percentile_support_types.
2017-09-25 09:28:42 -07:00
Tathagata Das f32a842505 [SPARK-22053][SS] Stream-stream inner join in Append Mode
## What changes were proposed in this pull request?

#### Architecture
This PR implements stream-stream inner join using a two-way symmetric hash join. At a high level, we want to do the following.

1. For each stream, we maintain the past rows as state in State Store.
  - For each joining key, there can be multiple rows that have been received.
  - So, we have to effectively maintain a key-to-list-of-values multimap as state for each stream.
2. In each batch, for each input row in each stream
  - Look up the other streams state to see if there are matching rows, and output them if they satisfy the joining condition
  - Add the input row to corresponding stream’s state.
  - If the data has a timestamp/window column with watermark, then we will use that to calculate the threshold for keys that are required to buffered for future matches and drop the rest from the state.

Cleaning up old unnecessary state rows depends completely on whether watermark has been defined and what are join conditions. We definitely want to support state clean up two types of queries that are likely to be common.

- Queries to time range conditions - E.g. `SELECT * FROM leftTable, rightTable ON leftKey = rightKey AND leftTime > rightTime - INTERVAL 8 MINUTES AND leftTime < rightTime + INTERVAL 1 HOUR`
- Queries with windows as the matching key - E.g. `SELECT * FROM leftTable, rightTable ON leftKey = rightKey AND window(leftTime, "1 hour") = window(rightTime, "1 hour")` (pseudo-SQL)

#### Implementation
The stream-stream join is primarily implemented in three classes
- `StreamingSymmetricHashJoinExec` implements the above symmetric join algorithm.
- `SymmetricsHashJoinStateManagers` manages the streaming state for the join. This essentially is a fault-tolerant key-to-list-of-values multimap built on the StateStore APIs. `StreamingSymmetricHashJoinExec` instantiates two such managers, one for each join side.
- `StreamingSymmetricHashJoinExecHelper` is a helper class to extract threshold for the state based on the join conditions and the event watermark.

Refer to the scaladocs class for more implementation details.

Besides the implementation of stream-stream inner join SparkPlan. Some additional changes are
- Allowed inner join in append mode in UnsupportedOperationChecker
- Prevented stream-stream join on an empty batch dataframe to be collapsed by the optimizer

## How was this patch tested?
- New tests in StreamingJoinSuite
- Updated tests UnsupportedOperationSuite

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

Closes #19271 from tdas/SPARK-22053.
2017-09-21 15:39:07 -07:00
Zhenhua Wang 1d1a09be9f [SPARK-17997][SQL] Add an aggregation function for counting distinct values for multiple intervals
## What changes were proposed in this pull request?

This work is a part of [SPARK-17074](https://issues.apache.org/jira/browse/SPARK-17074) to compute equi-height histograms. Equi-height histogram is an array of bins. A bin consists of two endpoints which form an interval of values and the ndv in that interval.

This PR creates a new aggregate function, given an array of endpoints, counting distinct values (ndv) in intervals among those endpoints.

This PR also refactors `HyperLogLogPlusPlus` by extracting a helper class `HyperLogLogPlusPlusHelper`, where the underlying HLLPP algorithm locates.

## How was this patch tested?

Add new test cases.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #15544 from wzhfy/countIntervals.
2017-09-21 21:43:02 +08:00
Kevin Yu c66d64b3df [SPARK-14878][SQL] Trim characters string function support
#### What changes were proposed in this pull request?

This PR enhances the TRIM function support in Spark SQL by allowing the specification
of trim characters set. Below is the SQL syntax :

``` SQL
<trim function> ::= TRIM <left paren> <trim operands> <right paren>
<trim operands> ::= [ [ <trim specification> ] [ <trim character set> ] FROM ] <trim source>
<trim source> ::= <character value expression>
<trim specification> ::=
  LEADING
| TRAILING
| BOTH
<trim character set> ::= <characters value expression>
```
or
``` SQL
LTRIM (source-exp [, trim-exp])
RTRIM (source-exp [, trim-exp])
```

Here are the documentation link of support of this feature by other mainstream databases.
- **Oracle:** [TRIM function](http://docs.oracle.com/cd/B28359_01/olap.111/b28126/dml_functions_2126.htm#OLADM704)
- **DB2:** [TRIM scalar function](https://www.ibm.com/support/knowledgecenter/en/SSMKHH_10.0.0/com.ibm.etools.mft.doc/ak05270_.htm)
- **MySQL:** [Trim function](http://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_trim)
- **Oracle:** [ltrim](https://docs.oracle.com/cd/B28359_01/olap.111/b28126/dml_functions_2018.htm#OLADM594)
- **DB2:** [ltrim](https://www.ibm.com/support/knowledgecenter/en/SSEPEK_11.0.0/sqlref/src/tpc/db2z_bif_ltrim.html)

This PR is to implement the above enhancement. In the implementation, the design principle is to keep the changes to the minimum. Also, the exiting trim functions (which handles a special case, i.e., trimming space characters) are kept unchanged for performane reasons.
#### How was this patch tested?

The unit test cases are added in the following files:
- UTF8StringSuite.java
- StringExpressionsSuite.scala
- sql/SQLQuerySuite.scala
- StringFunctionsSuite.scala

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

Closes #12646 from kevinyu98/spark-14878.
2017-09-18 12:12:35 -07:00
Tathagata Das 88661747f5 [SPARK-22018][SQL] Preserve top-level alias metadata when collapsing projects
## What changes were proposed in this pull request?
If there are two projects like as follows.
```
Project [a_with_metadata#27 AS b#26]
+- Project [a#0 AS a_with_metadata#27]
   +- LocalRelation <empty>, [a#0, b#1]
```
Child Project has an output column with a metadata in it, and the parent Project has an alias that implicitly forwards the metadata. So this metadata is visible for higher operators. Upon applying CollapseProject optimizer rule, the metadata is not preserved.
```
Project [a#0 AS b#26]
+- LocalRelation <empty>, [a#0, b#1]
```
This is incorrect, as downstream operators that expect certain metadata (e.g. watermark in structured streaming) to identify certain fields will fail to do so. This PR fixes it by preserving the metadata of top-level aliases.

## How was this patch tested?
New unit test

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

Closes #19240 from tdas/SPARK-22018.
2017-09-14 22:32:16 -07:00
goldmedal 371e4e2053 [SPARK-21513][SQL] Allow UDF to_json support converting MapType to json
# What changes were proposed in this pull request?
UDF to_json only supports converting `StructType` or `ArrayType` of `StructType`s to a json output string now.
According to the discussion of JIRA SPARK-21513, I allow to `to_json` support converting `MapType` and `ArrayType` of `MapType`s to a json output string.
This PR is for SQL and Scala API only.

# How was this patch tested?
Adding unit test case.

cc viirya HyukjinKwon

Author: goldmedal <liugs963@gmail.com>
Author: Jia-Xuan Liu <liugs963@gmail.com>

Closes #18875 from goldmedal/SPARK-21513.
2017-09-13 09:43:00 +09:00
Wang Gengliang 1a98574766 [SPARK-21979][SQL] Improve QueryPlanConstraints framework
## What changes were proposed in this pull request?

Improve QueryPlanConstraints framework, make it robust and simple.
In https://github.com/apache/spark/pull/15319, constraints for expressions like `a = f(b, c)` is resolved.
However, for expressions like
```scala
a = f(b, c) && c = g(a, b)
```
The current QueryPlanConstraints framework will produce non-converging constraints.
Essentially, the problem is caused by having both the name and child of aliases in the same constraint set.   We infer constraints, and push down constraints as predicates in filters, later on these predicates are propagated as constraints, etc..
Simply using the alias names only can resolve these problems.  The size of constraints is reduced without losing any information. We can always get these inferred constraints on child of aliases when pushing down filters.

Also, the EqualNullSafe between name and child in propagating alias is meaningless
```scala
allConstraints += EqualNullSafe(e, a.toAttribute)
```
It just produces redundant constraints.

## How was this patch tested?

Unit test

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #19201 from gengliangwang/QueryPlanConstraints.
2017-09-12 13:02:29 -07:00
Liang-Chi Hsieh 6b45d7e941 [SPARK-21954][SQL] JacksonUtils should verify MapType's value type instead of key type
## What changes were proposed in this pull request?

`JacksonUtils.verifySchema` verifies if a data type can be converted to JSON. For `MapType`, it now verifies the key type. However, in `JacksonGenerator`, when converting a map to JSON, we only care about its values and create a writer for the values. The keys in a map are treated as strings by calling `toString` on the keys.

Thus, we should change `JacksonUtils.verifySchema` to verify the value type of `MapType`.

## How was this patch tested?

Added tests.

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

Closes #19167 from viirya/test-jacksonutils.
2017-09-09 19:10:52 +09:00
Liang-Chi Hsieh 6e37524a1f [SPARK-21726][SQL] Check for structural integrity of the plan in Optimzer in test mode.
## What changes were proposed in this pull request?

We have many optimization rules now in `Optimzer`. Right now we don't have any checks in the optimizer to check for the structural integrity of the plan (e.g. resolved). When debugging, it is difficult to identify which rules return invalid plans.

It would be great if in test mode, we can check whether a plan is still resolved after the execution of each rule, so we can catch rules that return invalid plans.

## How was this patch tested?

Added tests.

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

Closes #18956 from viirya/SPARK-21726.
2017-09-07 23:12:18 -07:00
Jose Torres acdf45fb52 [SPARK-21765] Check that optimization doesn't affect isStreaming bit.
## What changes were proposed in this pull request?

Add an assert in logical plan optimization that the isStreaming bit stays the same, and fix empty relation rules where that wasn't happening.

## How was this patch tested?

new and existing unit tests

Author: Jose Torres <joseph.torres@databricks.com>
Author: Jose Torres <joseph-torres@databricks.com>

Closes #19056 from joseph-torres/SPARK-21765-followup.
2017-09-06 11:19:46 -07:00
Liang-Chi Hsieh 9f30d92803 [SPARK-21654][SQL] Complement SQL predicates expression description
## What changes were proposed in this pull request?

SQL predicates don't have complete expression description. This patch goes to complement the description by adding arguments, examples.

This change also adds related test cases for the SQL predicate expressions.

## How was this patch tested?

Existing tests. And added predicate test.

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

Closes #18869 from viirya/SPARK-21654.
2017-09-03 21:55:18 -07:00
Sean Owen 12ab7f7e89 [SPARK-14280][BUILD][WIP] Update change-version.sh and pom.xml to add Scala 2.12 profiles and enable 2.12 compilation
…build; fix some things that will be warnings or errors in 2.12; restore Scala 2.12 profile infrastructure

## What changes were proposed in this pull request?

This change adds back the infrastructure for a Scala 2.12 build, but does not enable it in the release or Python test scripts.

In order to make that meaningful, it also resolves compile errors that the code hits in 2.12 only, in a way that still works with 2.11.

It also updates dependencies to the earliest minor release of dependencies whose current version does not yet support Scala 2.12. This is in a sense covered by other JIRAs under the main umbrella, but implemented here. The versions below still work with 2.11, and are the _latest_ maintenance release in the _earliest_ viable minor release.

- Scalatest 2.x -> 3.0.3
- Chill 0.8.0 -> 0.8.4
- Clapper 1.0.x -> 1.1.2
- json4s 3.2.x -> 3.4.2
- Jackson 2.6.x -> 2.7.9 (required by json4s)

This change does _not_ fully enable a Scala 2.12 build:

- It will also require dropping support for Kafka before 0.10. Easy enough, just didn't do it yet here
- It will require recreating `SparkILoop` and `Main` for REPL 2.12, which is SPARK-14650. Possible to do here too.

What it does do is make changes that resolve much of the remaining gap without affecting the current 2.11 build.

## How was this patch tested?

Existing tests and build. Manually tested with `./dev/change-scala-version.sh 2.12` to verify it compiles, modulo the exceptions above.

Author: Sean Owen <sowen@cloudera.com>

Closes #18645 from srowen/SPARK-14280.
2017-09-01 19:21:21 +01:00
Andrew Ray cba69aeb45 [SPARK-21110][SQL] Structs, arrays, and other orderable datatypes should be usable in inequalities
## What changes were proposed in this pull request?

Allows `BinaryComparison` operators to work on any data type that actually supports ordering as verified by `TypeUtils.checkForOrderingExpr` instead of relying on the incomplete list `TypeCollection.Ordered` (which is removed by this PR).

## How was this patch tested?

Updated unit tests to cover structs and arrays.

Author: Andrew Ray <ray.andrew@gmail.com>

Closes #18818 from aray/SPARK-21110.
2017-08-31 15:08:03 -07:00
Herman van Hovell 05af2de0fd [SPARK-21830][SQL] Bump ANTLR version and fix a few issues.
## What changes were proposed in this pull request?
This PR bumps the ANTLR version to 4.7, and fixes a number of small parser related issues uncovered by the bump.

The main reason for upgrading is that in some cases the current version of ANTLR (4.5) can exhibit exponential slowdowns if it needs to parse boolean predicates. For example the following query will take forever to parse:
```sql
SELECT *
FROM RANGE(1000)
WHERE
TRUE
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
```

This is caused by a know bug in ANTLR (https://github.com/antlr/antlr4/issues/994), which was fixed in version 4.6.

## How was this patch tested?
Existing tests.

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

Closes #19042 from hvanhovell/SPARK-21830.
2017-08-24 16:33:55 -07:00
Liang-Chi Hsieh 183d4cb71f [SPARK-21759][SQL] In.checkInputDataTypes should not wrongly report unresolved plans for IN correlated subquery
## What changes were proposed in this pull request?

With the check for structural integrity proposed in SPARK-21726, it is found that the optimization rule `PullupCorrelatedPredicates` can produce unresolved plans.

For a correlated IN query looks like:

    SELECT t1.a FROM t1
    WHERE
    t1.a IN (SELECT t2.c
            FROM t2
            WHERE t1.b < t2.d);

The query plan might look like:

    Project [a#0]
    +- Filter a#0 IN (list#4 [b#1])
       :  +- Project [c#2]
       :     +- Filter (outer(b#1) < d#3)
       :        +- LocalRelation <empty>, [c#2, d#3]
       +- LocalRelation <empty>, [a#0, b#1]

After `PullupCorrelatedPredicates`, it produces query plan like:

    'Project [a#0]
    +- 'Filter a#0 IN (list#4 [(b#1 < d#3)])
       :  +- Project [c#2, d#3]
       :     +- LocalRelation <empty>, [c#2, d#3]
       +- LocalRelation <empty>, [a#0, b#1]

Because the correlated predicate involves another attribute `d#3` in subquery, it has been pulled out and added into the `Project` on the top of the subquery.

When `list` in `In` contains just one `ListQuery`, `In.checkInputDataTypes` checks if the size of `value` expressions matches the output size of subquery. In the above example, there is only `value` expression and the subquery output has two attributes `c#2, d#3`, so it fails the check and `In.resolved` returns `false`.

We should not let `In.checkInputDataTypes` wrongly report unresolved plans to fail the structural integrity check.

## How was this patch tested?

Added test.

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

Closes #18968 from viirya/SPARK-21759.
2017-08-24 21:46:58 +08:00
Jen-Ming Chung 95713eb4f2 [SPARK-21804][SQL] json_tuple returns null values within repeated columns except the first one
## What changes were proposed in this pull request?

When json_tuple in extracting values from JSON it returns null values within repeated columns except the first one as below:

``` scala
scala> spark.sql("""SELECT json_tuple('{"a":1, "b":2}', 'a', 'b', 'a')""").show()
+---+---+----+
| c0| c1|  c2|
+---+---+----+
|  1|  2|null|
+---+---+----+
```

I think this should be consistent with Hive's implementation:
```
hive> SELECT json_tuple('{"a": 1, "b": 2}', 'a', 'a');
...
1    1
```

In this PR, we located all the matched indices in `fieldNames` instead of returning the first matched index, i.e., indexOf.

## How was this patch tested?

Added test in JsonExpressionsSuite.

Author: Jen-Ming Chung <jenmingisme@gmail.com>

Closes #19017 from jmchung/SPARK-21804.
2017-08-24 19:24:00 +09:00
10129659 b8aaef49fb [SPARK-21807][SQL] Override ++ operation in ExpressionSet to reduce clone time
## What changes were proposed in this pull request?
The getAliasedConstraints  fuction in LogicalPlan.scala will clone the expression set when an element added,
and it will take a long time. This PR add a function to add multiple elements at once to reduce the clone time.

Before modified, the cost of getAliasedConstraints is:
100 expressions:  41 seconds
150 expressions:  466 seconds

After modified, the cost of getAliasedConstraints is:
100 expressions:  1.8 seconds
150 expressions:  6.5 seconds

The test is like this:
test("getAliasedConstraints") {
    val expressionNum = 150
    val aggExpression = (1 to expressionNum).map(i => Alias(Count(Literal(1)), s"cnt$i")())
    val aggPlan = Aggregate(Nil, aggExpression, LocalRelation())

    val beginTime = System.currentTimeMillis()
    val expressions = aggPlan.validConstraints
    println(s"validConstraints cost: ${System.currentTimeMillis() - beginTime}ms")
    // The size of Aliased expression is n * (n - 1) / 2 + n
    assert( expressions.size === expressionNum * (expressionNum - 1) / 2 + expressionNum)
  }

(Please fill in changes proposed in this fix)

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Run new added test.

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

Author: 10129659 <chen.yanshan@zte.com.cn>

Closes #19022 from eatoncys/getAliasedConstraints.
2017-08-23 20:35:08 -07:00
Jose Torres 3c0c2d09ca [SPARK-21765] Set isStreaming on leaf nodes for streaming plans.
## What changes were proposed in this pull request?
All streaming logical plans will now have isStreaming set. This involved adding isStreaming as a case class arg in a few cases, since a node might be logically streaming depending on where it came from.

## How was this patch tested?

Existing unit tests - no functional change is intended in this PR.

Author: Jose Torres <joseph-torres@databricks.com>
Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #18973 from joseph-torres/SPARK-21765.
2017-08-22 19:07:43 -07:00
Jen-Ming Chung 7ab951885f [SPARK-21677][SQL] json_tuple throws NullPointException when column is null as string type
## What changes were proposed in this pull request?
``` scala
scala> Seq(("""{"Hyukjin": 224, "John": 1225}""")).toDS.selectExpr("json_tuple(value, trim(null))").show()
...
java.lang.NullPointerException
	at ...
```

Currently the `null` field name will throw NullPointException. As a given field name null can't be matched with any field names in json, we just output null as its column value. This PR achieves it by returning a very unlikely column name `__NullFieldName` in evaluation of the field names.

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

Author: Jen-Ming Chung <jenmingisme@gmail.com>

Closes #18930 from jmchung/SPARK-21677.
2017-08-17 15:59:45 -07:00
Takeshi Yamamuro 6aad02d036 [SPARK-18394][SQL] Make an AttributeSet.toSeq output order consistent
## What changes were proposed in this pull request?
This pr sorted output attributes on their name and exprId in `AttributeSet.toSeq` to make the order consistent.  If the order is different, spark possibly generates different code and then misses cache in `CodeGenerator`, e.g., `GenerateColumnAccessor` generates code depending on an input attribute order.

## How was this patch tested?
Added tests in `AttributeSetSuite` and manually checked if the cache worked well in the given query of the JIRA.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #18959 from maropu/SPARK-18394.
2017-08-17 22:47:14 +02:00
10129659 1cce1a3b63 [SPARK-21603][SQL] The wholestage codegen will be much slower then that is closed when the function is too long
## What changes were proposed in this pull request?
Close the whole stage codegen when the function lines is longer than the maxlines which will be setted by
spark.sql.codegen.MaxFunctionLength parameter, because when the function is too long , it will not get the JIT  optimizing.
A benchmark test result is 10x slower when the generated function is too long :

ignore("max function length of wholestagecodegen") {
    val N = 20 << 15

    val benchmark = new Benchmark("max function length of wholestagecodegen", N)
    def f(): Unit = sparkSession.range(N)
      .selectExpr(
        "id",
        "(id & 1023) as k1",
        "cast(id & 1023 as double) as k2",
        "cast(id & 1023 as int) as k3",
        "case when id > 100 and id <= 200 then 1 else 0 end as v1",
        "case when id > 200 and id <= 300 then 1 else 0 end as v2",
        "case when id > 300 and id <= 400 then 1 else 0 end as v3",
        "case when id > 400 and id <= 500 then 1 else 0 end as v4",
        "case when id > 500 and id <= 600 then 1 else 0 end as v5",
        "case when id > 600 and id <= 700 then 1 else 0 end as v6",
        "case when id > 700 and id <= 800 then 1 else 0 end as v7",
        "case when id > 800 and id <= 900 then 1 else 0 end as v8",
        "case when id > 900 and id <= 1000 then 1 else 0 end as v9",
        "case when id > 1000 and id <= 1100 then 1 else 0 end as v10",
        "case when id > 1100 and id <= 1200 then 1 else 0 end as v11",
        "case when id > 1200 and id <= 1300 then 1 else 0 end as v12",
        "case when id > 1300 and id <= 1400 then 1 else 0 end as v13",
        "case when id > 1400 and id <= 1500 then 1 else 0 end as v14",
        "case when id > 1500 and id <= 1600 then 1 else 0 end as v15",
        "case when id > 1600 and id <= 1700 then 1 else 0 end as v16",
        "case when id > 1700 and id <= 1800 then 1 else 0 end as v17",
        "case when id > 1800 and id <= 1900 then 1 else 0 end as v18")
      .groupBy("k1", "k2", "k3")
      .sum()
      .collect()

    benchmark.addCase(s"codegen = F") { iter =>
      sparkSession.conf.set("spark.sql.codegen.wholeStage", "false")
      f()
    }

    benchmark.addCase(s"codegen = T") { iter =>
      sparkSession.conf.set("spark.sql.codegen.wholeStage", "true")
      sparkSession.conf.set("spark.sql.codegen.MaxFunctionLength", "10000")
      f()
    }

    benchmark.run()

    /*
    Java HotSpot(TM) 64-Bit Server VM 1.8.0_111-b14 on Windows 7 6.1
    Intel64 Family 6 Model 58 Stepping 9, GenuineIntel
    max function length of wholestagecodegen: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    ------------------------------------------------------------------------------------------------
    codegen = F                                    443 /  507          1.5         676.0       1.0X
    codegen = T                                   3279 / 3283          0.2        5002.6       0.1X
     */
  }

## How was this patch tested?
Run the unit test

Author: 10129659 <chen.yanshan@zte.com.cn>

Closes #18810 from eatoncys/codegen.
2017-08-16 09:12:20 -07:00
Marcelo Vanzin 3f958a9992 [SPARK-21731][BUILD] Upgrade scalastyle to 0.9.
This version fixes a few issues in the import order checker; it provides
better error messages, and detects more improper ordering (thus the need
to change a lot of files in this patch). The main fix is that it correctly
complains about the order of packages vs. classes.

As part of the above, I moved some "SparkSession" import in ML examples
inside the "$example on$" blocks; that didn't seem consistent across
different source files to start with, and avoids having to add more on/off blocks
around specific imports.

The new scalastyle also seems to have a better header detector, so a few
license headers had to be updated to match the expected indentation.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #18943 from vanzin/SPARK-21731.
2017-08-15 13:59:00 -07:00
Wenchen Fan 14bdb25fd7 [SPARK-18464][SQL][FOLLOWUP] support old table which doesn't store schema in table properties
## What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/15900 , to fix one more bug:
When table schema is empty and need to be inferred at runtime, we should not resolve parent plans before the schema has been inferred, or the parent plans will be resolved against an empty schema and may get wrong result for something like `select *`

The fix logic is: introduce `UnresolvedCatalogRelation` as a placeholder. Then we replace it with `LogicalRelation` or `HiveTableRelation` during analysis, so that it's guaranteed that we won't resolve parent plans until the schema has been inferred.

## How was this patch tested?

regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18907 from cloud-fan/bug.
2017-08-15 09:04:56 -07:00
aokolnychyi 5596ce83c4 [MINOR][SQL] Additional test case for CheckCartesianProducts rule
## What changes were proposed in this pull request?

While discovering optimization rules and their test coverage, I did not find any tests for `CheckCartesianProducts` in the Catalyst folder. So, I decided to create a new test suite. Once I finished, I found a test in `JoinSuite` for this functionality so feel free to discard this change if it does not make much sense. The proposed test suite covers a few additional use cases.

Author: aokolnychyi <anton.okolnychyi@sap.com>

Closes #18909 from aokolnychyi/check-cartesian-join-tests.
2017-08-13 21:33:16 -07:00
Reynold Xin 584c7f1437 [SPARK-21699][SQL] Remove unused getTableOption in ExternalCatalog
## What changes were proposed in this pull request?
This patch removes the unused SessionCatalog.getTableMetadataOption and ExternalCatalog. getTableOption.

## How was this patch tested?
Removed the test case.

Author: Reynold Xin <rxin@databricks.com>

Closes #18912 from rxin/remove-getTableOption.
2017-08-10 18:56:25 -07:00
Jose Torres 0fb73253fc [SPARK-21587][SS] Added filter pushdown through watermarks.
## What changes were proposed in this pull request?

Push filter predicates through EventTimeWatermark if they're deterministic and do not reference the watermarked attribute. (This is similar but not identical to the logic for pushing through UnaryNode.)

## How was this patch tested?
unit tests

Author: Jose Torres <joseph-torres@databricks.com>

Closes #18790 from joseph-torres/SPARK-21587.
2017-08-09 12:50:04 -07:00
gatorsmile 2d799d0808 [SPARK-21504][SQL] Add spark version info into table metadata
## What changes were proposed in this pull request?
This PR is to add the spark version info in the table metadata. When creating the table, this value is assigned. It can help users find which version of Spark was used to create the table.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18709 from gatorsmile/addVersion.
2017-08-09 08:46:25 -07:00
BartekH 438c381584 Add "full_outer" name to join types
I have discovered that "full_outer" name option is working in Spark 2.0, but it is not printed in exception. Please verify.

## What changes were proposed in this pull request?

(Please fill in changes proposed in this fix)

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

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

Author: BartekH <bartekhamielec@gmail.com>

Closes #17985 from BartekH/patch-1.
2017-08-06 16:40:59 -07:00
Takeshi Yamamuro 74b47845ea [SPARK-20963][SQL][FOLLOW-UP] Use UnresolvedSubqueryColumnAliases for visitTableName
## What changes were proposed in this pull request?
This pr (follow-up of #18772) used `UnresolvedSubqueryColumnAliases` for `visitTableName` in `AstBuilder`, which is a new unresolved `LogicalPlan` implemented in #18185.

## How was this patch tested?
Existing tests

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #18857 from maropu/SPARK-20963-FOLLOWUP.
2017-08-06 10:14:45 -07:00
Takeshi Yamamuro 990efad1c6 [SPARK-20963][SQL] Support column aliases for join relations in FROM clause
## What changes were proposed in this pull request?
This pr added parsing rules to support column aliases for join relations in FROM clause.
This pr is a sub-task of #18079.

## How was this patch tested?
Added tests in `AnalysisSuite`, `PlanParserSuite,` and `SQLQueryTestSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #18772 from maropu/SPARK-20963-2.
2017-08-05 20:35:54 -07:00
Reynold Xin 5ad1796b9f [SPARK-21634][SQL] Change OneRowRelation from a case object to case class
## What changes were proposed in this pull request?
OneRowRelation is the only plan that is a case object, which causes some issues with makeCopy using a 0-arg constructor. This patch changes it from a case object to a case class.

This blocks SPARK-21619.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #18839 from rxin/SPARK-21634.
2017-08-04 10:36:08 -07:00
Yuming Wang 231f67247b [SPARK-21205][SQL] pmod(number, 0) should be null.
## What changes were proposed in this pull request?
Hive `pmod(3.13, 0)`:
```:sql
hive> select pmod(3.13, 0);
OK
NULL
Time taken: 2.514 seconds, Fetched: 1 row(s)
hive>
```

Spark `mod(3.13, 0)`:
```:sql
spark-sql> select mod(3.13, 0);
NULL
spark-sql>
```

But the Spark `pmod(3.13, 0)`:
```:sql
spark-sql> select pmod(3.13, 0);
17/06/25 09:35:58 ERROR SparkSQLDriver: Failed in [select pmod(3.13, 0)]
java.lang.NullPointerException
	at org.apache.spark.sql.catalyst.expressions.Pmod.pmod(arithmetic.scala:504)
	at org.apache.spark.sql.catalyst.expressions.Pmod.nullSafeEval(arithmetic.scala:432)
	at org.apache.spark.sql.catalyst.expressions.BinaryExpression.eval(Expression.scala:419)
	at org.apache.spark.sql.catalyst.expressions.UnaryExpression.eval(Expression.scala:323)
...
```
This PR make `pmod(number, 0)` to null.

## How was this patch tested?
unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18413 from wangyum/SPARK-21205.
2017-08-04 12:06:08 +02:00
bravo-zhang 6b186c9d60 [SPARK-18950][SQL] Report conflicting fields when merging two StructTypes
## What changes were proposed in this pull request?

Currently, StructType.merge() only reports data types of conflicting fields when merging two incompatible schemas. It would be nice to also report the field names for easier debugging.

## How was this patch tested?

Unit test in DataTypeSuite.
Print exception message when conflict is triggered.

Author: bravo-zhang <mzhang1230@gmail.com>

Closes #16365 from bravo-zhang/spark-18950.
2017-07-31 17:19:55 -07:00
Takeshi Yamamuro 6550086bbd [SPARK-20962][SQL] Support subquery column aliases in FROM clause
## What changes were proposed in this pull request?
This pr added parsing rules to support subquery column aliases in FROM clause.
This pr is a sub-task of #18079.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #18185 from maropu/SPARK-20962.
2017-07-29 10:14:47 -07:00
Xingbo Jiang 92d85637e7 [SPARK-19451][SQL] rangeBetween method should accept Long value as boundary
## What changes were proposed in this pull request?

Long values can be passed to `rangeBetween` as range frame boundaries, but we silently convert it to Int values, this can cause wrong results and we should fix this.

Further more, we should accept any legal literal values as range frame boundaries. In this PR, we make it possible for Long values, and make accepting other DataTypes really easy to add.

This PR is mostly based on Herman's previous amazing work: 596f53c339

After this been merged, we can close #16818 .

## How was this patch tested?

Add new tests in `DataFrameWindowFunctionsSuite` and `TypeCoercionSuite`.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #18540 from jiangxb1987/rangeFrame.
2017-07-29 10:11:31 -07:00
pj.fanning 2a53fbfce7 [SPARK-20871][SQL] limit logging of Janino code
## What changes were proposed in this pull request?

When the code that is generated is greater than 64k, then Janino compile will fail and CodeGenerator.scala will log the entire code at Error level.
SPARK-20871 suggests only logging the code at Debug level.
Since, the code is already logged at debug level, this Pull Request proposes not including the formatted code in the Error logging and exception message at all.
When an exception occurs, the code will be logged at Info level but truncated if it is more than 1000 lines long.

## How was this patch tested?

Existing tests were run.
An extra test test case was added to CodeFormatterSuite to test the new maxLines parameter,

Author: pj.fanning <pj.fanning@workday.com>

Closes #18658 from pjfanning/SPARK-20871.
2017-07-23 10:38:03 -07:00
gatorsmile ae253e5a87 [SPARK-21273][SQL][FOLLOW-UP] Propagate logical plan stats using visitor pattern and mixin
## What changes were proposed in this pull request?
This PR is to add back the stats propagation of `Window` and remove the stats calculation of the leaf node `Range`, which has been covered by 9c32d2507d/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala (L56)

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18677 from gatorsmile/visitStats.
2017-07-19 10:57:15 +08:00
Wenchen Fan f18b905f6c [SPARK-21457][SQL] ExternalCatalog.listPartitions should correctly handle partition values with dot
## What changes were proposed in this pull request?

When we list partitions from hive metastore with a partial partition spec, we are expecting exact matching according to the partition values. However, hive treats dot specially and match any single character for dot. We should do an extra filter to drop unexpected partitions.

## How was this patch tested?

new regression test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18671 from cloud-fan/hive.
2017-07-18 15:56:16 -07:00
Sean Owen e26dac5feb [SPARK-21415] Triage scapegoat warnings, part 1
## What changes were proposed in this pull request?

Address scapegoat warnings for:
- BigDecimal double constructor
- Catching NPE
- Finalizer without super
- List.size is O(n)
- Prefer Seq.empty
- Prefer Set.empty
- reverse.map instead of reverseMap
- Type shadowing
- Unnecessary if condition.
- Use .log1p
- Var could be val

In some instances like Seq.empty, I avoided making the change even where valid in test code to keep the scope of the change smaller. Those issues are concerned with performance and it won't matter for tests.

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #18635 from srowen/Scapegoat1.
2017-07-18 08:47:17 +01:00
aokolnychyi 0be5fb41a6 [SPARK-21332][SQL] Incorrect result type inferred for some decimal expressions
## What changes were proposed in this pull request?

This PR changes the direction of expression transformation in the DecimalPrecision rule. Previously, the expressions were transformed down, which led to incorrect result types when decimal expressions had other decimal expressions as their operands. The root cause of this issue was in visiting outer nodes before their children. Consider the example below:

```
    val inputSchema = StructType(StructField("col", DecimalType(26, 6)) :: Nil)
    val sc = spark.sparkContext
    val rdd = sc.parallelize(1 to 2).map(_ => Row(BigDecimal(12)))
    val df = spark.createDataFrame(rdd, inputSchema)

    // Works correctly since no nested decimal expression is involved
    // Expected result type: (26, 6) * (26, 6) = (38, 12)
    df.select($"col" * $"col").explain(true)
    df.select($"col" * $"col").printSchema()

    // Gives a wrong result since there is a nested decimal expression that should be visited first
    // Expected result type: ((26, 6) * (26, 6)) * (26, 6) = (38, 12) * (26, 6) = (38, 18)
    df.select($"col" * $"col" * $"col").explain(true)
    df.select($"col" * $"col" * $"col").printSchema()
```

The example above gives the following output:

```
// Correct result without sub-expressions
== Parsed Logical Plan ==
'Project [('col * 'col) AS (col * col)#4]
+- LogicalRDD [col#1]

== Analyzed Logical Plan ==
(col * col): decimal(38,12)
Project [CheckOverflow((promote_precision(cast(col#1 as decimal(26,6))) * promote_precision(cast(col#1 as decimal(26,6)))), DecimalType(38,12)) AS (col * col)#4]
+- LogicalRDD [col#1]

== Optimized Logical Plan ==
Project [CheckOverflow((col#1 * col#1), DecimalType(38,12)) AS (col * col)#4]
+- LogicalRDD [col#1]

== Physical Plan ==
*Project [CheckOverflow((col#1 * col#1), DecimalType(38,12)) AS (col * col)#4]
+- Scan ExistingRDD[col#1]

// Schema
root
 |-- (col * col): decimal(38,12) (nullable = true)

// Incorrect result with sub-expressions
== Parsed Logical Plan ==
'Project [(('col * 'col) * 'col) AS ((col * col) * col)#11]
+- LogicalRDD [col#1]

== Analyzed Logical Plan ==
((col * col) * col): decimal(38,12)
Project [CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(col#1 as decimal(26,6))) * promote_precision(cast(col#1 as decimal(26,6)))), DecimalType(38,12)) as decimal(26,6))) * promote_precision(cast(col#1 as decimal(26,6)))), DecimalType(38,12)) AS ((col * col) * col)#11]
+- LogicalRDD [col#1]

== Optimized Logical Plan ==
Project [CheckOverflow((cast(CheckOverflow((col#1 * col#1), DecimalType(38,12)) as decimal(26,6)) * col#1), DecimalType(38,12)) AS ((col * col) * col)#11]
+- LogicalRDD [col#1]

== Physical Plan ==
*Project [CheckOverflow((cast(CheckOverflow((col#1 * col#1), DecimalType(38,12)) as decimal(26,6)) * col#1), DecimalType(38,12)) AS ((col * col) * col)#11]
+- Scan ExistingRDD[col#1]

// Schema
root
 |-- ((col * col) * col): decimal(38,12) (nullable = true)
```

## How was this patch tested?

This PR was tested with available unit tests. Moreover, there are tests to cover previously failing scenarios.

Author: aokolnychyi <anton.okolnychyi@sap.com>

Closes #18583 from aokolnychyi/spark-21332.
2017-07-17 21:07:50 -07:00
Sean Owen fd52a747fd [SPARK-19810][SPARK-19810][MINOR][FOLLOW-UP] Follow-ups from to remove Scala 2.10
## What changes were proposed in this pull request?

Follow up to a few comments on https://github.com/apache/spark/pull/17150#issuecomment-315020196 that couldn't be addressed before it was merged.

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #18646 from srowen/SPARK-19810.2.
2017-07-17 09:22:42 +08:00
Kazuaki Ishizaki ac5d5d7959 [SPARK-21344][SQL] BinaryType comparison does signed byte array comparison
## What changes were proposed in this pull request?

This PR fixes a wrong comparison for `BinaryType`. This PR enables unsigned comparison and unsigned prefix generation for an array for `BinaryType`. Previous implementations uses signed operations.

## How was this patch tested?

Added a test suite in `OrderingSuite`.

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

Closes #18571 from kiszk/SPARK-21344.
2017-07-14 20:16:04 -07:00
Sean Owen 425c4ada4c [SPARK-19810][BUILD][CORE] Remove support for Scala 2.10
## What changes were proposed in this pull request?

- Remove Scala 2.10 build profiles and support
- Replace some 2.10 support in scripts with commented placeholders for 2.12 later
- Remove deprecated API calls from 2.10 support
- Remove usages of deprecated context bounds where possible
- Remove Scala 2.10 workarounds like ScalaReflectionLock
- Other minor Scala warning fixes

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #17150 from srowen/SPARK-19810.
2017-07-13 17:06:24 +08:00
Takeshi Yamamuro 647963a26a [SPARK-20460][SQL] Make it more consistent to handle column name duplication
## What changes were proposed in this pull request?
This pr made it more consistent to handle column name duplication. In the current master, error handling is different when hitting column name duplication:
```
// json
scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil)
scala> Seq("""{"a":1, "a":1}"""""").toDF().coalesce(1).write.mode("overwrite").text("/tmp/data")
scala> spark.read.format("json").schema(schema).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Reference 'a' is ambiguous, could be: a#12, a#13.;
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:287)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:181)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:153)

scala> spark.read.format("json").load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Duplicate column(s) : "a" found, cannot save to JSON format;
  at org.apache.spark.sql.execution.datasources.json.JsonDataSource.checkConstraints(JsonDataSource.scala:81)
  at org.apache.spark.sql.execution.datasources.json.JsonDataSource.inferSchema(JsonDataSource.scala:63)
  at org.apache.spark.sql.execution.datasources.json.JsonFileFormat.inferSchema(JsonFileFormat.scala:57)
  at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$7.apply(DataSource.scala:176)
  at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$7.apply(DataSource.scala:176)

// csv
scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil)
scala> Seq("a,a", "1,1").toDF().coalesce(1).write.mode("overwrite").text("/tmp/data")
scala> spark.read.format("csv").schema(schema).option("header", false).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Reference 'a' is ambiguous, could be: a#41, a#42.;
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:287)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:181)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:153)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:152)

// If `inferSchema` is true, a CSV format is duplicate-safe (See SPARK-16896)
scala> spark.read.format("csv").option("header", true).load("/tmp/data").show
+---+---+
| a0| a1|
+---+---+
|  1|  1|
+---+---+

// parquet
scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil)
scala> Seq((1, 1)).toDF("a", "b").coalesce(1).write.mode("overwrite").parquet("/tmp/data")
scala> spark.read.format("parquet").schema(schema).option("header", false).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Reference 'a' is ambiguous, could be: a#110, a#111.;
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:287)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:181)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:153)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:152)
  at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
  at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
```
When this patch applied, the results change to;
```

// json
scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil)
scala> Seq("""{"a":1, "a":1}"""""").toDF().coalesce(1).write.mode("overwrite").text("/tmp/data")
scala> spark.read.format("json").schema(schema).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in datasource: "a";
  at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtil.scala:47)
  at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtil.scala:33)
  at org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:186)
  at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:368)

scala> spark.read.format("json").load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in datasource: "a";
  at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtil.scala:47)
  at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtil.scala:33)
  at org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:186)
  at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:368)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:178)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:156)

// csv
scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil)
scala> Seq("a,a", "1,1").toDF().coalesce(1).write.mode("overwrite").text("/tmp/data")
scala> spark.read.format("csv").schema(schema).option("header", false).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in datasource: "a";
  at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtil.scala:47)
  at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtil.scala:33)
  at org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:186)
  at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:368)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:178)

scala> spark.read.format("csv").option("header", true).load("/tmp/data").show
+---+---+
| a0| a1|
+---+---+
|  1|  1|
+---+---+

// parquet
scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil)
scala> Seq((1, 1)).toDF("a", "b").coalesce(1).write.mode("overwrite").parquet("/tmp/data")
scala> spark.read.format("parquet").schema(schema).option("header", false).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in datasource: "a";
  at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtil.scala:47)
  at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtil.scala:33)
  at org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:186)
  at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:368)
```

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17758 from maropu/SPARK-20460.
2017-07-10 15:58:34 +08:00
Xiao Li c3712b77a9 [SPARK-21307][REVERT][SQL] Remove SQLConf parameters from the parser-related classes
## What changes were proposed in this pull request?
Since we do not set active sessions when parsing the plan, we are unable to correctly use SQLConf.get to find the correct active session. Since https://github.com/apache/spark/pull/18531 breaks the build, I plan to revert it at first.

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18568 from gatorsmile/revert18531.
2017-07-08 11:56:19 -07:00
Takeshi Yamamuro 7896e7b99d [SPARK-21281][SQL] Use string types by default if array and map have no argument
## What changes were proposed in this pull request?
This pr modified code to use string types by default if `array` and `map` in functions have no argument. This behaviour is the same with Hive one;
```
hive> CREATE TEMPORARY TABLE t1 AS SELECT map();
hive> DESCRIBE t1;
_c0   map<string,string>

hive> CREATE TEMPORARY TABLE t2 AS SELECT array();
hive> DESCRIBE t2;
_c0   array<string>
```

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #18516 from maropu/SPARK-21281.
2017-07-07 23:05:38 -07:00
Wenchen Fan fef081309f [SPARK-21335][SQL] support un-aliased subquery
## What changes were proposed in this pull request?

un-aliased subquery is supported by Spark SQL for a long time. Its semantic was not well defined and had confusing behaviors, and it's not a standard SQL syntax, so we disallowed it in https://issues.apache.org/jira/browse/SPARK-20690 .

However, this is a breaking change, and we do have existing queries using un-aliased subquery. We should add the support back and fix its semantic.

This PR fixes the un-aliased subquery by assigning a default alias name.

After this PR, there is no syntax change from branch 2.2 to master, but we invalid a weird use case:
`SELECT v.i from (SELECT i FROM v)`. Now this query will throw analysis exception because users should not be able to use the qualifier inside a subquery.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18559 from cloud-fan/sub-query.
2017-07-07 20:04:30 +08:00
Bogdan Raducanu 26ac085deb [SPARK-21228][SQL] InSet incorrect handling of structs
## What changes were proposed in this pull request?
When data type is struct, InSet now uses TypeUtils.getInterpretedOrdering (similar to EqualTo) to build a TreeSet. In other cases it will use a HashSet as before (which should be faster). Similarly, In.eval uses Ordering.equiv instead of equals.

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

Author: Bogdan Raducanu <bogdan@databricks.com>

Closes #18455 from bogdanrdc/SPARK-21228.
2017-07-07 01:04:57 +08:00
Wang Gengliang d540dfbff3 [SPARK-21273][SQL][FOLLOW-UP] Add missing test cases back and revise code style
## What changes were proposed in this pull request?

Add missing test cases back and revise code style

Follow up the previous PR: https://github.com/apache/spark/pull/18479

## How was this patch tested?

Unit test

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

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #18548 from gengliangwang/stat_propagation_revise.
2017-07-06 19:12:15 +08:00
gatorsmile 75b168fd30 [SPARK-21308][SQL] Remove SQLConf parameters from the optimizer
### What changes were proposed in this pull request?
This PR removes SQLConf parameters from the optimizer rules

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18533 from gatorsmile/rmSQLConfOptimizer.
2017-07-06 14:18:50 +08:00
gatorsmile c8e7f445b9 [SPARK-21307][SQL] Remove SQLConf parameters from the parser-related classes.
### What changes were proposed in this pull request?
This PR is to remove SQLConf parameters from the parser-related classes.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18531 from gatorsmile/rmSQLConfParser.
2017-07-05 11:06:15 -07:00
ouyangxiaochen 5787ace463 [SPARK-20383][SQL] Supporting Create [temporary] Function with the keyword 'OR REPLACE' and 'IF NOT EXISTS'
## What changes were proposed in this pull request?

support to create [temporary] function with the keyword 'OR REPLACE' and 'IF NOT EXISTS'

## How was this patch tested?
manual test and added test cases

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

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

Closes #17681 from ouyangxiaochen/spark-419.
2017-07-05 20:46:42 +08:00
Takuya UESHIN 873f3ad2b8 [SPARK-16167][SQL] RowEncoder should preserve array/map type nullability.
## What changes were proposed in this pull request?

Currently `RowEncoder` doesn't preserve nullability of `ArrayType` or `MapType`.
It returns always `containsNull = true` for `ArrayType`, `valueContainsNull = true` for `MapType` and also the nullability of itself is always `true`.

This pr fixes the nullability of them.
## How was this patch tested?

Add tests to check if `RowEncoder` preserves array/map nullability.

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

Closes #13873 from ueshin/issues/SPARK-16167.
2017-07-05 20:32:47 +08:00
Takuya UESHIN ce10545d34 [SPARK-21300][SQL] ExternalMapToCatalyst should null-check map key prior to converting to internal value.
## What changes were proposed in this pull request?

`ExternalMapToCatalyst` should null-check map key prior to converting to internal value to throw an appropriate Exception instead of something like NPE.

## How was this patch tested?

Added a test and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #18524 from ueshin/issues/SPARK-21300.
2017-07-05 11:24:38 +08:00
gatorsmile 29b1f6b09f [SPARK-21256][SQL] Add withSQLConf to Catalyst Test
### What changes were proposed in this pull request?
SQLConf is moved to Catalyst. We are adding more and more test cases for verifying the conf-specific behaviors. It is nice to add a helper function to simplify the test cases.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18469 from gatorsmile/withSQLConf.
2017-07-04 08:54:07 -07:00
Wenchen Fan f953ca56ec [SPARK-21284][SQL] rename SessionCatalog.registerFunction parameter name
## What changes were proposed in this pull request?

Looking at the code in `SessionCatalog.registerFunction`, the parameter `ignoreIfExists` is a wrong name. When `ignoreIfExists` is true, we will override the function if it already exists. So `overrideIfExists` should be the corrected name.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18510 from cloud-fan/minor.
2017-07-03 10:51:44 -07:00
Reynold Xin b1d719e7c9 [SPARK-21273][SQL] Propagate logical plan stats using visitor pattern and mixin
## What changes were proposed in this pull request?
We currently implement statistics propagation directly in logical plan. Given we already have two different implementations, it'd make sense to actually decouple the two and add stats propagation using mixin. This would reduce the coupling between logical plan and statistics handling.

This can also be a powerful pattern in the future to add additional properties (e.g. constraints).

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

Author: Reynold Xin <rxin@databricks.com>

Closes #18479 from rxin/stats-trait.
2017-06-30 21:10:23 -07:00
Wenchen Fan 4eb41879ce [SPARK-17528][SQL] data should be copied properly before saving into InternalRow
## What changes were proposed in this pull request?

For performance reasons, `UnsafeRow.getString`, `getStruct`, etc. return a "pointer" that points to a memory region of this unsafe row. This makes the unsafe projection a little dangerous, because all of its output rows share one instance.

When we implement SQL operators, we should be careful to not cache the input rows because they may be produced by unsafe projection from child operator and thus its content may change overtime.

However, when we updating values of InternalRow(e.g. in mutable projection and safe projection), we only copy UTF8String, we should also copy InternalRow, ArrayData and MapData. This PR fixes this, and also fixes the copy of vairous InternalRow, ArrayData and MapData implementations.

## How was this patch tested?

new regression tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18483 from cloud-fan/fix-copy.
2017-07-01 09:25:29 +08:00
Xiao Li eed9c4ef85 [SPARK-21129][SQL] Arguments of SQL function call should not be named expressions
### What changes were proposed in this pull request?

Function argument should not be named expressions. It could cause two issues:
- Misleading error message
- Unexpected query results when the column name is `distinct`, which is not a reserved word in our parser.

```
spark-sql> select count(distinct c1, distinct c2) from t1;
Error in query: cannot resolve '`distinct`' given input columns: [c1, c2]; line 1 pos 26;
'Project [unresolvedalias('count(c1#30, 'distinct), None)]
+- SubqueryAlias t1
   +- CatalogRelation `default`.`t1`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [c1#30, c2#31]
```

After the fix, the error message becomes
```
spark-sql> select count(distinct c1, distinct c2) from t1;
Error in query:
extraneous input 'c2' expecting {')', ',', '.', '[', 'OR', 'AND', 'IN', NOT, 'BETWEEN', 'LIKE', RLIKE, 'IS', EQ, '<=>', '<>', '!=', '<', LTE, '>', GTE, '+', '-', '*', '/', '%', 'DIV', '&', '|', '||', '^'}(line 1, pos 35)

== SQL ==
select count(distinct c1, distinct c2) from t1
-----------------------------------^^^
```

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

Author: Xiao Li <gatorsmile@gmail.com>
Author: gatorsmile <gatorsmile@gmail.com>

Closes #18338 from gatorsmile/parserDistinctAggFunc.
2017-06-30 14:23:56 -07:00
wangzhenhua 82e24912d6 [SPARK-21237][SQL] Invalidate stats once table data is changed
## What changes were proposed in this pull request?

Invalidate spark's stats after data changing commands:

- InsertIntoHadoopFsRelationCommand
- InsertIntoHiveTable
- LoadDataCommand
- TruncateTableCommand
- AlterTableSetLocationCommand
- AlterTableDropPartitionCommand

## How was this patch tested?

Added test cases.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #18449 from wzhfy/removeStats.
2017-06-29 11:32:29 +08:00
Wang Gengliang b72b8521d9 [SPARK-21222] Move elimination of Distinct clause from analyzer to optimizer
## What changes were proposed in this pull request?

Move elimination of Distinct clause from analyzer to optimizer

Distinct clause is useless after MAX/MIN clause. For example,
"Select MAX(distinct a) FROM src from"
is equivalent of
"Select MAX(a) FROM src from"
However, this optimization is implemented in analyzer. It should be in optimizer.

## How was this patch tested?

Unit test

gatorsmile cloud-fan

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

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #18429 from gengliangwang/distinct_opt.
2017-06-29 08:47:31 +08:00
Xiao Li 03eb6117af [SPARK-21164][SQL] Remove isTableSample from Sample and isGenerated from Alias and AttributeReference
## What changes were proposed in this pull request?
`isTableSample` and `isGenerated ` were introduced for SQL Generation respectively by https://github.com/apache/spark/pull/11148 and https://github.com/apache/spark/pull/11050

Since SQL Generation is removed, we do not need to keep `isTableSample`.

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18379 from gatorsmile/CleanSample.
2017-06-23 14:48:33 -07:00
Dilip Biswal 13c2a4f2f8 [SPARK-20417][SQL] Move subquery error handling to checkAnalysis from Analyzer
## What changes were proposed in this pull request?
Currently we do a lot of validations for subquery in the Analyzer. We should move them to CheckAnalysis which is the framework to catch and report Analysis errors. This was mentioned as a review comment in SPARK-18874.

## How was this patch tested?
Exists tests + A few tests added to SQLQueryTestSuite.

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

Closes #17713 from dilipbiswal/subquery_checkanalysis.
2017-06-23 11:02:54 -07:00
wangzhenhua b803b66a81 [SPARK-21180][SQL] Remove conf from stats functions since now we have conf in LogicalPlan
## What changes were proposed in this pull request?

After wiring `SQLConf` in logical plan ([PR 18299](https://github.com/apache/spark/pull/18299)), we can remove the need of passing `conf` into `def stats` and `def computeStats`.

## How was this patch tested?

Covered by existing tests, plus some modified existing tests.

Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>

Closes #18391 from wzhfy/removeConf.
2017-06-23 10:33:53 -07:00
Xingbo Jiang cad88f17e8 [SPARK-17851][SQL][TESTS] Make sure all test sqls in catalyst pass checkAnalysis
## What changes were proposed in this pull request?

Currently we have several tens of test sqls in catalyst will fail at `SimpleAnalyzer.checkAnalysis`, we should make sure they are valid.

This PR makes the following changes:
1. Apply `checkAnalysis` on plans that tests `Optimizer` rules, but don't require the testcases for `Parser`/`Analyzer` pass `checkAnalysis`;
2. Fix testcases for `Optimizer` that would have fall.
## How was this patch tested?

Apply `SimpleAnalyzer.checkAnalysis` on plans in `PlanTest.comparePlans`, update invalid test cases.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>
Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15417 from jiangxb1987/cptest.
2017-06-21 09:40:06 -07:00
Xiao Li 9413b84b5a [SPARK-21132][SQL] DISTINCT modifier of function arguments should not be silently ignored
### What changes were proposed in this pull request?
We should not silently ignore `DISTINCT` when they are not supported in the function arguments. This PR is to block these cases and issue the error messages.

### How was this patch tested?
Added test cases for both regular functions and window functions

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18340 from gatorsmile/firstCount.
2017-06-19 15:51:21 +08:00
Yuming Wang f913f158ec [SPARK-20948][SQL] Built-in SQL Function UnaryMinus/UnaryPositive support string type
## What changes were proposed in this pull request?

Built-in SQL Function UnaryMinus/UnaryPositive support string type, if it's string type, convert it to double type, after this PR:
```sql
spark-sql> select positive('-1.11'), negative('-1.11');
-1.11   1.11
spark-sql>
```

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18173 from wangyum/SPARK-20948.
2017-06-18 20:14:05 -07:00
Yuming Wang 53e48f73e4 [SPARK-20931][SQL] ABS function support string type.
## What changes were proposed in this pull request?

ABS function support string type. Hive/MySQL support this feature.

Ref: 4ba713ccd8/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAbs.java (L93)

## How was this patch tested?
 unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18153 from wangyum/SPARK-20931.
2017-06-16 09:40:58 -07:00
Kazuaki Ishizaki 7a3e5dc28b [SPARK-20749][SQL] Built-in SQL Function Support - all variants of LEN[GTH]
## What changes were proposed in this pull request?

This PR adds built-in SQL function `BIT_LENGTH()`, `CHAR_LENGTH()`, and `OCTET_LENGTH()` functions.

`BIT_LENGTH()` returns the bit length of the given string or binary expression.
`CHAR_LENGTH()` returns the length of the given string or binary expression. (i.e. equal to `LENGTH()`)
`OCTET_LENGTH()` returns the byte length of the given string or binary expression.

## How was this patch tested?

Added new test suites for these three functions

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

Closes #18046 from kiszk/SPARK-20749.
2017-06-15 23:06:58 -07:00
Xianyang Liu 87ab0cec65 [SPARK-21072][SQL] TreeNode.mapChildren should only apply to the children node.
## What changes were proposed in this pull request?

Just as the function name and comments of `TreeNode.mapChildren` mentioned, the function should be apply to all currently node children. So, the follow code should judge whether it is the children node.

https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala#L342

## How was this patch tested?

Existing tests.

Author: Xianyang Liu <xianyang.liu@intel.com>

Closes #18284 from ConeyLiu/treenode.
2017-06-16 12:10:09 +08:00
ALeksander Eskilson b32b2123dd [SPARK-18016][SQL][CATALYST] Code Generation: Constant Pool Limit - Class Splitting
## What changes were proposed in this pull request?

This pull-request exclusively includes the class splitting feature described in #16648. When code for a given class would grow beyond 1600k bytes, a private, nested sub-class is generated into which subsequent functions are inlined. Additional sub-classes are generated as the code threshold is met subsequent times. This code includes 3 changes:

1. Includes helper maps, lists, and functions for keeping track of sub-classes during code generation (included in the `CodeGenerator` class). These helper functions allow nested classes and split functions to be initialized/declared/inlined to the appropriate locations in the various projection classes.
2. Changes `addNewFunction` to return a string to support instances where a split function is inlined to a nested class and not the outer class (and so must be invoked using the class-qualified name). Uses of `addNewFunction` throughout the codebase are modified so that the returned name is properly used.
3. Removes instances of the `this` keyword when used on data inside generated classes. All state declared in the outer class is by default global and accessible to the nested classes. However, if a reference to global state in a nested class is prepended with the `this` keyword, it would attempt to reference state belonging to the nested class (which would not exist), rather than the correct variable belonging to the outer class.

## How was this patch tested?

Added a test case to the `GeneratedProjectionSuite` that increases the number of columns tested in various projections to a threshold that would previously have triggered a `JaninoRuntimeException` for the Constant Pool.

Note: This PR does not address the second Constant Pool issue with code generation (also mentioned in #16648): excess global mutable state. A second PR may be opened to resolve that issue.

Author: ALeksander Eskilson <alek.eskilson@cerner.com>

Closes #18075 from bdrillard/class_splitting_only.
2017-06-15 13:45:08 +08:00
Reynold Xin fffeb6d7c3 [SPARK-21092][SQL] Wire SQLConf in logical plan and expressions
## What changes were proposed in this pull request?
It is really painful to not have configs in logical plan and expressions. We had to add all sorts of hacks (e.g. pass SQLConf explicitly in functions). This patch exposes SQLConf in logical plan, using a thread local variable and a getter closure that's set once there is an active SparkSession.

The implementation is a bit of a hack, since we didn't anticipate this need in the beginning (config was only exposed in physical plan). The implementation is described in `SQLConf.get`.

In terms of future work, we should follow up to clean up CBO (remove the need for passing in config).

## How was this patch tested?
Updated relevant tests for constraint propagation.

Author: Reynold Xin <rxin@databricks.com>

Closes #18299 from rxin/SPARK-21092.
2017-06-14 22:11:41 -07:00
Yuming Wang 4d01aa4648 [SPARK-20754][SQL][FOLLOWUP] Add Function Alias For MOD/POSITION.
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/18106 Support TRUNC (number),  We should also add function alias for `MOD `and `POSITION`.

`POSITION(substr IN str) `is a synonym for `LOCATE(substr,str)`. same as MySQL: https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_position

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18206 from wangyum/SPARK-20754-mod&position.
2017-06-13 23:39:06 -07:00
Dongjoon Hyun 2639c3ed03 [SPARK-19910][SQL] stack should not reject NULL values due to type mismatch
## What changes were proposed in this pull request?

Since `stack` function generates a table with nullable columns, it should allow mixed null values.

```scala
scala> sql("select stack(3, 1, 2, 3)").printSchema
root
 |-- col0: integer (nullable = true)

scala> sql("select stack(3, 1, 2, null)").printSchema
org.apache.spark.sql.AnalysisException: cannot resolve 'stack(3, 1, 2, NULL)' due to data type mismatch: Argument 1 (IntegerType) != Argument 3 (NullType); line 1 pos 7;
```

## How was this patch tested?

Pass the Jenkins with a new test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #17251 from dongjoon-hyun/SPARK-19910.
2017-06-12 21:18:43 -07:00
Reynold Xin b1436c7496 [SPARK-21059][SQL] LikeSimplification can NPE on null pattern
## What changes were proposed in this pull request?
This patch fixes a bug that can cause NullPointerException in LikeSimplification, when the pattern for like is null.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #18273 from rxin/SPARK-21059.
2017-06-12 14:07:51 -07:00
aokolnychyi ca4e960aec [SPARK-17914][SQL] Fix parsing of timestamp strings with nanoseconds
The PR contains a tiny change to fix the way Spark parses string literals into timestamps. Currently, some timestamps that contain nanoseconds are corrupted during the conversion from internal UTF8Strings into the internal representation of timestamps.

Consider the following example:
```
spark.sql("SELECT cast('2015-01-02 00:00:00.000000001' as TIMESTAMP)").show(false)
+------------------------------------------------+
|CAST(2015-01-02 00:00:00.000000001 AS TIMESTAMP)|
+------------------------------------------------+
|2015-01-02 00:00:00.000001                      |
+------------------------------------------------+
```

The fix was tested with existing tests. Also, there is a new test to cover cases that did not work previously.

Author: aokolnychyi <anton.okolnychyi@sap.com>

Closes #18252 from aokolnychyi/spark-17914.
2017-06-12 13:06:14 -07:00
liuxian d140918093 [SPARK-20665][SQL][FOLLOW-UP] Move test case to MathExpressionsSuite
## What changes were proposed in this pull request?

 add test case to MathExpressionsSuite as #17906

## How was this patch tested?

unit test cases

Author: liuxian <liu.xian3@zte.com.cn>

Closes #18082 from 10110346/wip-lx-0524.
2017-06-11 22:29:09 -07:00
Michal Senkyr 0538f3b0ae [SPARK-18891][SQL] Support for Scala Map collection types
## What changes were proposed in this pull request?

Add support for arbitrary Scala `Map` types in deserialization as well as a generic implicit encoder.

Used the builder approach as in #16541 to construct any provided `Map` type upon deserialization.

Please note that this PR also adds (ignored) tests for issue [SPARK-19104 CompileException with Map and Case Class in Spark 2.1.0](https://issues.apache.org/jira/browse/SPARK-19104) but doesn't solve it.

Added support for Java Maps in codegen code (encoders will be added in a different PR) with the following default implementations for interfaces/abstract classes:

* `java.util.Map`, `java.util.AbstractMap` => `java.util.HashMap`
* `java.util.SortedMap`, `java.util.NavigableMap` => `java.util.TreeMap`
* `java.util.concurrent.ConcurrentMap` => `java.util.concurrent.ConcurrentHashMap`
* `java.util.concurrent.ConcurrentNavigableMap` => `java.util.concurrent.ConcurrentSkipListMap`

Resulting codegen for `Seq(Map(1 -> 2)).toDS().map(identity).queryExecution.debug.codegen`:

```
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIterator(references);
/* 003 */ }
/* 004 */
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private scala.collection.Iterator inputadapter_input;
/* 009 */   private boolean CollectObjectsToMap_loopIsNull1;
/* 010 */   private int CollectObjectsToMap_loopValue0;
/* 011 */   private boolean CollectObjectsToMap_loopIsNull3;
/* 012 */   private int CollectObjectsToMap_loopValue2;
/* 013 */   private UnsafeRow deserializetoobject_result;
/* 014 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder deserializetoobject_holder;
/* 015 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter deserializetoobject_rowWriter;
/* 016 */   private scala.collection.immutable.Map mapelements_argValue;
/* 017 */   private UnsafeRow mapelements_result;
/* 018 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder mapelements_holder;
/* 019 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter mapelements_rowWriter;
/* 020 */   private UnsafeRow serializefromobject_result;
/* 021 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 022 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 023 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter serializefromobject_arrayWriter;
/* 024 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter serializefromobject_arrayWriter1;
/* 025 */
/* 026 */   public GeneratedIterator(Object[] references) {
/* 027 */     this.references = references;
/* 028 */   }
/* 029 */
/* 030 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 031 */     partitionIndex = index;
/* 032 */     this.inputs = inputs;
/* 033 */     wholestagecodegen_init_0();
/* 034 */     wholestagecodegen_init_1();
/* 035 */
/* 036 */   }
/* 037 */
/* 038 */   private void wholestagecodegen_init_0() {
/* 039 */     inputadapter_input = inputs[0];
/* 040 */
/* 041 */     deserializetoobject_result = new UnsafeRow(1);
/* 042 */     this.deserializetoobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(deserializetoobject_result, 32);
/* 043 */     this.deserializetoobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(deserializetoobject_holder, 1);
/* 044 */
/* 045 */     mapelements_result = new UnsafeRow(1);
/* 046 */     this.mapelements_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(mapelements_result, 32);
/* 047 */     this.mapelements_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(mapelements_holder, 1);
/* 048 */     serializefromobject_result = new UnsafeRow(1);
/* 049 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 32);
/* 050 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 051 */     this.serializefromobject_arrayWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
/* 052 */
/* 053 */   }
/* 054 */
/* 055 */   private void wholestagecodegen_init_1() {
/* 056 */     this.serializefromobject_arrayWriter1 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
/* 057 */
/* 058 */   }
/* 059 */
/* 060 */   protected void processNext() throws java.io.IOException {
/* 061 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 062 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 063 */       boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 064 */       MapData inputadapter_value = inputadapter_isNull ? null : (inputadapter_row.getMap(0));
/* 065 */
/* 066 */       boolean deserializetoobject_isNull1 = true;
/* 067 */       ArrayData deserializetoobject_value1 = null;
/* 068 */       if (!inputadapter_isNull) {
/* 069 */         deserializetoobject_isNull1 = false;
/* 070 */         if (!deserializetoobject_isNull1) {
/* 071 */           Object deserializetoobject_funcResult = null;
/* 072 */           deserializetoobject_funcResult = inputadapter_value.keyArray();
/* 073 */           if (deserializetoobject_funcResult == null) {
/* 074 */             deserializetoobject_isNull1 = true;
/* 075 */           } else {
/* 076 */             deserializetoobject_value1 = (ArrayData) deserializetoobject_funcResult;
/* 077 */           }
/* 078 */
/* 079 */         }
/* 080 */         deserializetoobject_isNull1 = deserializetoobject_value1 == null;
/* 081 */       }
/* 082 */
/* 083 */       boolean deserializetoobject_isNull3 = true;
/* 084 */       ArrayData deserializetoobject_value3 = null;
/* 085 */       if (!inputadapter_isNull) {
/* 086 */         deserializetoobject_isNull3 = false;
/* 087 */         if (!deserializetoobject_isNull3) {
/* 088 */           Object deserializetoobject_funcResult1 = null;
/* 089 */           deserializetoobject_funcResult1 = inputadapter_value.valueArray();
/* 090 */           if (deserializetoobject_funcResult1 == null) {
/* 091 */             deserializetoobject_isNull3 = true;
/* 092 */           } else {
/* 093 */             deserializetoobject_value3 = (ArrayData) deserializetoobject_funcResult1;
/* 094 */           }
/* 095 */
/* 096 */         }
/* 097 */         deserializetoobject_isNull3 = deserializetoobject_value3 == null;
/* 098 */       }
/* 099 */       scala.collection.immutable.Map deserializetoobject_value = null;
/* 100 */
/* 101 */       if ((deserializetoobject_isNull1 && !deserializetoobject_isNull3) ||
/* 102 */         (!deserializetoobject_isNull1 && deserializetoobject_isNull3)) {
/* 103 */         throw new RuntimeException("Invalid state: Inconsistent nullability of key-value");
/* 104 */       }
/* 105 */
/* 106 */       if (!deserializetoobject_isNull1) {
/* 107 */         if (deserializetoobject_value1.numElements() != deserializetoobject_value3.numElements()) {
/* 108 */           throw new RuntimeException("Invalid state: Inconsistent lengths of key-value arrays");
/* 109 */         }
/* 110 */         int deserializetoobject_dataLength = deserializetoobject_value1.numElements();
/* 111 */
/* 112 */         scala.collection.mutable.Builder CollectObjectsToMap_builderValue5 = scala.collection.immutable.Map$.MODULE$.newBuilder();
/* 113 */         CollectObjectsToMap_builderValue5.sizeHint(deserializetoobject_dataLength);
/* 114 */
/* 115 */         int deserializetoobject_loopIndex = 0;
/* 116 */         while (deserializetoobject_loopIndex < deserializetoobject_dataLength) {
/* 117 */           CollectObjectsToMap_loopValue0 = (int) (deserializetoobject_value1.getInt(deserializetoobject_loopIndex));
/* 118 */           CollectObjectsToMap_loopValue2 = (int) (deserializetoobject_value3.getInt(deserializetoobject_loopIndex));
/* 119 */           CollectObjectsToMap_loopIsNull1 = deserializetoobject_value1.isNullAt(deserializetoobject_loopIndex);
/* 120 */           CollectObjectsToMap_loopIsNull3 = deserializetoobject_value3.isNullAt(deserializetoobject_loopIndex);
/* 121 */
/* 122 */           if (CollectObjectsToMap_loopIsNull1) {
/* 123 */             throw new RuntimeException("Found null in map key!");
/* 124 */           }
/* 125 */
/* 126 */           scala.Tuple2 CollectObjectsToMap_loopValue4;
/* 127 */
/* 128 */           if (CollectObjectsToMap_loopIsNull3) {
/* 129 */             CollectObjectsToMap_loopValue4 = new scala.Tuple2(CollectObjectsToMap_loopValue0, null);
/* 130 */           } else {
/* 131 */             CollectObjectsToMap_loopValue4 = new scala.Tuple2(CollectObjectsToMap_loopValue0, CollectObjectsToMap_loopValue2);
/* 132 */           }
/* 133 */
/* 134 */           CollectObjectsToMap_builderValue5.$plus$eq(CollectObjectsToMap_loopValue4);
/* 135 */
/* 136 */           deserializetoobject_loopIndex += 1;
/* 137 */         }
/* 138 */
/* 139 */         deserializetoobject_value = (scala.collection.immutable.Map) CollectObjectsToMap_builderValue5.result();
/* 140 */       }
/* 141 */
/* 142 */       boolean mapelements_isNull = true;
/* 143 */       scala.collection.immutable.Map mapelements_value = null;
/* 144 */       if (!false) {
/* 145 */         mapelements_argValue = deserializetoobject_value;
/* 146 */
/* 147 */         mapelements_isNull = false;
/* 148 */         if (!mapelements_isNull) {
/* 149 */           Object mapelements_funcResult = null;
/* 150 */           mapelements_funcResult = ((scala.Function1) references[0]).apply(mapelements_argValue);
/* 151 */           if (mapelements_funcResult == null) {
/* 152 */             mapelements_isNull = true;
/* 153 */           } else {
/* 154 */             mapelements_value = (scala.collection.immutable.Map) mapelements_funcResult;
/* 155 */           }
/* 156 */
/* 157 */         }
/* 158 */         mapelements_isNull = mapelements_value == null;
/* 159 */       }
/* 160 */
/* 161 */       MapData serializefromobject_value = null;
/* 162 */       if (!mapelements_isNull) {
/* 163 */         final int serializefromobject_length = mapelements_value.size();
/* 164 */         final Object[] serializefromobject_convertedKeys = new Object[serializefromobject_length];
/* 165 */         final Object[] serializefromobject_convertedValues = new Object[serializefromobject_length];
/* 166 */         int serializefromobject_index = 0;
/* 167 */         final scala.collection.Iterator serializefromobject_entries = mapelements_value.iterator();
/* 168 */         while(serializefromobject_entries.hasNext()) {
/* 169 */           final scala.Tuple2 serializefromobject_entry = (scala.Tuple2) serializefromobject_entries.next();
/* 170 */           int ExternalMapToCatalyst_key1 = (Integer) serializefromobject_entry._1();
/* 171 */           int ExternalMapToCatalyst_value1 = (Integer) serializefromobject_entry._2();
/* 172 */
/* 173 */           boolean ExternalMapToCatalyst_value_isNull1 = false;
/* 174 */
/* 175 */           if (false) {
/* 176 */             throw new RuntimeException("Cannot use null as map key!");
/* 177 */           } else {
/* 178 */             serializefromobject_convertedKeys[serializefromobject_index] = (Integer) ExternalMapToCatalyst_key1;
/* 179 */           }
/* 180 */
/* 181 */           if (false) {
/* 182 */             serializefromobject_convertedValues[serializefromobject_index] = null;
/* 183 */           } else {
/* 184 */             serializefromobject_convertedValues[serializefromobject_index] = (Integer) ExternalMapToCatalyst_value1;
/* 185 */           }
/* 186 */
/* 187 */           serializefromobject_index++;
/* 188 */         }
/* 189 */
/* 190 */         serializefromobject_value = new org.apache.spark.sql.catalyst.util.ArrayBasedMapData(new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_convertedKeys), new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_convertedValues));
/* 191 */       }
/* 192 */       serializefromobject_holder.reset();
/* 193 */
/* 194 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 195 */
/* 196 */       if (mapelements_isNull) {
/* 197 */         serializefromobject_rowWriter.setNullAt(0);
/* 198 */       } else {
/* 199 */         // Remember the current cursor so that we can calculate how many bytes are
/* 200 */         // written later.
/* 201 */         final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
/* 202 */
/* 203 */         if (serializefromobject_value instanceof UnsafeMapData) {
/* 204 */           final int serializefromobject_sizeInBytes = ((UnsafeMapData) serializefromobject_value).getSizeInBytes();
/* 205 */           // grow the global buffer before writing data.
/* 206 */           serializefromobject_holder.grow(serializefromobject_sizeInBytes);
/* 207 */           ((UnsafeMapData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 208 */           serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
/* 209 */
/* 210 */         } else {
/* 211 */           final ArrayData serializefromobject_keys = serializefromobject_value.keyArray();
/* 212 */           final ArrayData serializefromobject_values = serializefromobject_value.valueArray();
/* 213 */
/* 214 */           // preserve 8 bytes to write the key array numBytes later.
/* 215 */           serializefromobject_holder.grow(8);
/* 216 */           serializefromobject_holder.cursor += 8;
/* 217 */
/* 218 */           // Remember the current cursor so that we can write numBytes of key array later.
/* 219 */           final int serializefromobject_tmpCursor1 = serializefromobject_holder.cursor;
/* 220 */
/* 221 */           if (serializefromobject_keys instanceof UnsafeArrayData) {
/* 222 */             final int serializefromobject_sizeInBytes1 = ((UnsafeArrayData) serializefromobject_keys).getSizeInBytes();
/* 223 */             // grow the global buffer before writing data.
/* 224 */             serializefromobject_holder.grow(serializefromobject_sizeInBytes1);
/* 225 */             ((UnsafeArrayData) serializefromobject_keys).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 226 */             serializefromobject_holder.cursor += serializefromobject_sizeInBytes1;
/* 227 */
/* 228 */           } else {
/* 229 */             final int serializefromobject_numElements = serializefromobject_keys.numElements();
/* 230 */             serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 4);
/* 231 */
/* 232 */             for (int serializefromobject_index1 = 0; serializefromobject_index1 < serializefromobject_numElements; serializefromobject_index1++) {
/* 233 */               if (serializefromobject_keys.isNullAt(serializefromobject_index1)) {
/* 234 */                 serializefromobject_arrayWriter.setNullInt(serializefromobject_index1);
/* 235 */               } else {
/* 236 */                 final int serializefromobject_element = serializefromobject_keys.getInt(serializefromobject_index1);
/* 237 */                 serializefromobject_arrayWriter.write(serializefromobject_index1, serializefromobject_element);
/* 238 */               }
/* 239 */             }
/* 240 */           }
/* 241 */
/* 242 */           // Write the numBytes of key array into the first 8 bytes.
/* 243 */           Platform.putLong(serializefromobject_holder.buffer, serializefromobject_tmpCursor1 - 8, serializefromobject_holder.cursor - serializefromobject_tmpCursor1);
/* 244 */
/* 245 */           if (serializefromobject_values instanceof UnsafeArrayData) {
/* 246 */             final int serializefromobject_sizeInBytes2 = ((UnsafeArrayData) serializefromobject_values).getSizeInBytes();
/* 247 */             // grow the global buffer before writing data.
/* 248 */             serializefromobject_holder.grow(serializefromobject_sizeInBytes2);
/* 249 */             ((UnsafeArrayData) serializefromobject_values).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 250 */             serializefromobject_holder.cursor += serializefromobject_sizeInBytes2;
/* 251 */
/* 252 */           } else {
/* 253 */             final int serializefromobject_numElements1 = serializefromobject_values.numElements();
/* 254 */             serializefromobject_arrayWriter1.initialize(serializefromobject_holder, serializefromobject_numElements1, 4);
/* 255 */
/* 256 */             for (int serializefromobject_index2 = 0; serializefromobject_index2 < serializefromobject_numElements1; serializefromobject_index2++) {
/* 257 */               if (serializefromobject_values.isNullAt(serializefromobject_index2)) {
/* 258 */                 serializefromobject_arrayWriter1.setNullInt(serializefromobject_index2);
/* 259 */               } else {
/* 260 */                 final int serializefromobject_element1 = serializefromobject_values.getInt(serializefromobject_index2);
/* 261 */                 serializefromobject_arrayWriter1.write(serializefromobject_index2, serializefromobject_element1);
/* 262 */               }
/* 263 */             }
/* 264 */           }
/* 265 */
/* 266 */         }
/* 267 */
/* 268 */         serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
/* 269 */       }
/* 270 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
/* 271 */       append(serializefromobject_result);
/* 272 */       if (shouldStop()) return;
/* 273 */     }
/* 274 */   }
/* 275 */ }
```

Codegen for `java.util.Map`:

```
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIterator(references);
/* 003 */ }
/* 004 */
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private scala.collection.Iterator inputadapter_input;
/* 009 */   private boolean CollectObjectsToMap_loopIsNull1;
/* 010 */   private int CollectObjectsToMap_loopValue0;
/* 011 */   private boolean CollectObjectsToMap_loopIsNull3;
/* 012 */   private int CollectObjectsToMap_loopValue2;
/* 013 */   private UnsafeRow deserializetoobject_result;
/* 014 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder deserializetoobject_holder;
/* 015 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter deserializetoobject_rowWriter;
/* 016 */   private java.util.HashMap mapelements_argValue;
/* 017 */   private UnsafeRow mapelements_result;
/* 018 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder mapelements_holder;
/* 019 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter mapelements_rowWriter;
/* 020 */   private UnsafeRow serializefromobject_result;
/* 021 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 022 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 023 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter serializefromobject_arrayWriter;
/* 024 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter serializefromobject_arrayWriter1;
/* 025 */
/* 026 */   public GeneratedIterator(Object[] references) {
/* 027 */     this.references = references;
/* 028 */   }
/* 029 */
/* 030 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 031 */     partitionIndex = index;
/* 032 */     this.inputs = inputs;
/* 033 */     wholestagecodegen_init_0();
/* 034 */     wholestagecodegen_init_1();
/* 035 */
/* 036 */   }
/* 037 */
/* 038 */   private void wholestagecodegen_init_0() {
/* 039 */     inputadapter_input = inputs[0];
/* 040 */
/* 041 */     deserializetoobject_result = new UnsafeRow(1);
/* 042 */     this.deserializetoobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(deserializetoobject_result, 32);
/* 043 */     this.deserializetoobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(deserializetoobject_holder, 1);
/* 044 */
/* 045 */     mapelements_result = new UnsafeRow(1);
/* 046 */     this.mapelements_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(mapelements_result, 32);
/* 047 */     this.mapelements_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(mapelements_holder, 1);
/* 048 */     serializefromobject_result = new UnsafeRow(1);
/* 049 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 32);
/* 050 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 051 */     this.serializefromobject_arrayWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
/* 052 */
/* 053 */   }
/* 054 */
/* 055 */   private void wholestagecodegen_init_1() {
/* 056 */     this.serializefromobject_arrayWriter1 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
/* 057 */
/* 058 */   }
/* 059 */
/* 060 */   protected void processNext() throws java.io.IOException {
/* 061 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 062 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 063 */       boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 064 */       MapData inputadapter_value = inputadapter_isNull ? null : (inputadapter_row.getMap(0));
/* 065 */
/* 066 */       boolean deserializetoobject_isNull1 = true;
/* 067 */       ArrayData deserializetoobject_value1 = null;
/* 068 */       if (!inputadapter_isNull) {
/* 069 */         deserializetoobject_isNull1 = false;
/* 070 */         if (!deserializetoobject_isNull1) {
/* 071 */           Object deserializetoobject_funcResult = null;
/* 072 */           deserializetoobject_funcResult = inputadapter_value.keyArray();
/* 073 */           if (deserializetoobject_funcResult == null) {
/* 074 */             deserializetoobject_isNull1 = true;
/* 075 */           } else {
/* 076 */             deserializetoobject_value1 = (ArrayData) deserializetoobject_funcResult;
/* 077 */           }
/* 078 */
/* 079 */         }
/* 080 */         deserializetoobject_isNull1 = deserializetoobject_value1 == null;
/* 081 */       }
/* 082 */
/* 083 */       boolean deserializetoobject_isNull3 = true;
/* 084 */       ArrayData deserializetoobject_value3 = null;
/* 085 */       if (!inputadapter_isNull) {
/* 086 */         deserializetoobject_isNull3 = false;
/* 087 */         if (!deserializetoobject_isNull3) {
/* 088 */           Object deserializetoobject_funcResult1 = null;
/* 089 */           deserializetoobject_funcResult1 = inputadapter_value.valueArray();
/* 090 */           if (deserializetoobject_funcResult1 == null) {
/* 091 */             deserializetoobject_isNull3 = true;
/* 092 */           } else {
/* 093 */             deserializetoobject_value3 = (ArrayData) deserializetoobject_funcResult1;
/* 094 */           }
/* 095 */
/* 096 */         }
/* 097 */         deserializetoobject_isNull3 = deserializetoobject_value3 == null;
/* 098 */       }
/* 099 */       java.util.HashMap deserializetoobject_value = null;
/* 100 */
/* 101 */       if ((deserializetoobject_isNull1 && !deserializetoobject_isNull3) ||
/* 102 */         (!deserializetoobject_isNull1 && deserializetoobject_isNull3)) {
/* 103 */         throw new RuntimeException("Invalid state: Inconsistent nullability of key-value");
/* 104 */       }
/* 105 */
/* 106 */       if (!deserializetoobject_isNull1) {
/* 107 */         if (deserializetoobject_value1.numElements() != deserializetoobject_value3.numElements()) {
/* 108 */           throw new RuntimeException("Invalid state: Inconsistent lengths of key-value arrays");
/* 109 */         }
/* 110 */         int deserializetoobject_dataLength = deserializetoobject_value1.numElements();
/* 111 */         java.util.Map CollectObjectsToMap_builderValue5 = new java.util.HashMap(deserializetoobject_dataLength);
/* 112 */
/* 113 */         int deserializetoobject_loopIndex = 0;
/* 114 */         while (deserializetoobject_loopIndex < deserializetoobject_dataLength) {
/* 115 */           CollectObjectsToMap_loopValue0 = (int) (deserializetoobject_value1.getInt(deserializetoobject_loopIndex));
/* 116 */           CollectObjectsToMap_loopValue2 = (int) (deserializetoobject_value3.getInt(deserializetoobject_loopIndex));
/* 117 */           CollectObjectsToMap_loopIsNull1 = deserializetoobject_value1.isNullAt(deserializetoobject_loopIndex);
/* 118 */           CollectObjectsToMap_loopIsNull3 = deserializetoobject_value3.isNullAt(deserializetoobject_loopIndex);
/* 119 */
/* 120 */           if (CollectObjectsToMap_loopIsNull1) {
/* 121 */             throw new RuntimeException("Found null in map key!");
/* 122 */           }
/* 123 */
/* 124 */           CollectObjectsToMap_builderValue5.put(CollectObjectsToMap_loopValue0, CollectObjectsToMap_loopValue2);
/* 125 */
/* 126 */           deserializetoobject_loopIndex += 1;
/* 127 */         }
/* 128 */
/* 129 */         deserializetoobject_value = (java.util.HashMap) CollectObjectsToMap_builderValue5;
/* 130 */       }
/* 131 */
/* 132 */       boolean mapelements_isNull = true;
/* 133 */       java.util.HashMap mapelements_value = null;
/* 134 */       if (!false) {
/* 135 */         mapelements_argValue = deserializetoobject_value;
/* 136 */
/* 137 */         mapelements_isNull = false;
/* 138 */         if (!mapelements_isNull) {
/* 139 */           Object mapelements_funcResult = null;
/* 140 */           mapelements_funcResult = ((scala.Function1) references[0]).apply(mapelements_argValue);
/* 141 */           if (mapelements_funcResult == null) {
/* 142 */             mapelements_isNull = true;
/* 143 */           } else {
/* 144 */             mapelements_value = (java.util.HashMap) mapelements_funcResult;
/* 145 */           }
/* 146 */
/* 147 */         }
/* 148 */         mapelements_isNull = mapelements_value == null;
/* 149 */       }
/* 150 */
/* 151 */       MapData serializefromobject_value = null;
/* 152 */       if (!mapelements_isNull) {
/* 153 */         final int serializefromobject_length = mapelements_value.size();
/* 154 */         final Object[] serializefromobject_convertedKeys = new Object[serializefromobject_length];
/* 155 */         final Object[] serializefromobject_convertedValues = new Object[serializefromobject_length];
/* 156 */         int serializefromobject_index = 0;
/* 157 */         final java.util.Iterator serializefromobject_entries = mapelements_value.entrySet().iterator();
/* 158 */         while(serializefromobject_entries.hasNext()) {
/* 159 */           final java.util.Map$Entry serializefromobject_entry = (java.util.Map$Entry) serializefromobject_entries.next();
/* 160 */           int ExternalMapToCatalyst_key1 = (Integer) serializefromobject_entry.getKey();
/* 161 */           int ExternalMapToCatalyst_value1 = (Integer) serializefromobject_entry.getValue();
/* 162 */
/* 163 */           boolean ExternalMapToCatalyst_value_isNull1 = false;
/* 164 */
/* 165 */           if (false) {
/* 166 */             throw new RuntimeException("Cannot use null as map key!");
/* 167 */           } else {
/* 168 */             serializefromobject_convertedKeys[serializefromobject_index] = (Integer) ExternalMapToCatalyst_key1;
/* 169 */           }
/* 170 */
/* 171 */           if (false) {
/* 172 */             serializefromobject_convertedValues[serializefromobject_index] = null;
/* 173 */           } else {
/* 174 */             serializefromobject_convertedValues[serializefromobject_index] = (Integer) ExternalMapToCatalyst_value1;
/* 175 */           }
/* 176 */
/* 177 */           serializefromobject_index++;
/* 178 */         }
/* 179 */
/* 180 */         serializefromobject_value = new org.apache.spark.sql.catalyst.util.ArrayBasedMapData(new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_convertedKeys), new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_convertedValues));
/* 181 */       }
/* 182 */       serializefromobject_holder.reset();
/* 183 */
/* 184 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 185 */
/* 186 */       if (mapelements_isNull) {
/* 187 */         serializefromobject_rowWriter.setNullAt(0);
/* 188 */       } else {
/* 189 */         // Remember the current cursor so that we can calculate how many bytes are
/* 190 */         // written later.
/* 191 */         final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
/* 192 */
/* 193 */         if (serializefromobject_value instanceof UnsafeMapData) {
/* 194 */           final int serializefromobject_sizeInBytes = ((UnsafeMapData) serializefromobject_value).getSizeInBytes();
/* 195 */           // grow the global buffer before writing data.
/* 196 */           serializefromobject_holder.grow(serializefromobject_sizeInBytes);
/* 197 */           ((UnsafeMapData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 198 */           serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
/* 199 */
/* 200 */         } else {
/* 201 */           final ArrayData serializefromobject_keys = serializefromobject_value.keyArray();
/* 202 */           final ArrayData serializefromobject_values = serializefromobject_value.valueArray();
/* 203 */
/* 204 */           // preserve 8 bytes to write the key array numBytes later.
/* 205 */           serializefromobject_holder.grow(8);
/* 206 */           serializefromobject_holder.cursor += 8;
/* 207 */
/* 208 */           // Remember the current cursor so that we can write numBytes of key array later.
/* 209 */           final int serializefromobject_tmpCursor1 = serializefromobject_holder.cursor;
/* 210 */
/* 211 */           if (serializefromobject_keys instanceof UnsafeArrayData) {
/* 212 */             final int serializefromobject_sizeInBytes1 = ((UnsafeArrayData) serializefromobject_keys).getSizeInBytes();
/* 213 */             // grow the global buffer before writing data.
/* 214 */             serializefromobject_holder.grow(serializefromobject_sizeInBytes1);
/* 215 */             ((UnsafeArrayData) serializefromobject_keys).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 216 */             serializefromobject_holder.cursor += serializefromobject_sizeInBytes1;
/* 217 */
/* 218 */           } else {
/* 219 */             final int serializefromobject_numElements = serializefromobject_keys.numElements();
/* 220 */             serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 4);
/* 221 */
/* 222 */             for (int serializefromobject_index1 = 0; serializefromobject_index1 < serializefromobject_numElements; serializefromobject_index1++) {
/* 223 */               if (serializefromobject_keys.isNullAt(serializefromobject_index1)) {
/* 224 */                 serializefromobject_arrayWriter.setNullInt(serializefromobject_index1);
/* 225 */               } else {
/* 226 */                 final int serializefromobject_element = serializefromobject_keys.getInt(serializefromobject_index1);
/* 227 */                 serializefromobject_arrayWriter.write(serializefromobject_index1, serializefromobject_element);
/* 228 */               }
/* 229 */             }
/* 230 */           }
/* 231 */
/* 232 */           // Write the numBytes of key array into the first 8 bytes.
/* 233 */           Platform.putLong(serializefromobject_holder.buffer, serializefromobject_tmpCursor1 - 8, serializefromobject_holder.cursor - serializefromobject_tmpCursor1);
/* 234 */
/* 235 */           if (serializefromobject_values instanceof UnsafeArrayData) {
/* 236 */             final int serializefromobject_sizeInBytes2 = ((UnsafeArrayData) serializefromobject_values).getSizeInBytes();
/* 237 */             // grow the global buffer before writing data.
/* 238 */             serializefromobject_holder.grow(serializefromobject_sizeInBytes2);
/* 239 */             ((UnsafeArrayData) serializefromobject_values).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 240 */             serializefromobject_holder.cursor += serializefromobject_sizeInBytes2;
/* 241 */
/* 242 */           } else {
/* 243 */             final int serializefromobject_numElements1 = serializefromobject_values.numElements();
/* 244 */             serializefromobject_arrayWriter1.initialize(serializefromobject_holder, serializefromobject_numElements1, 4);
/* 245 */
/* 246 */             for (int serializefromobject_index2 = 0; serializefromobject_index2 < serializefromobject_numElements1; serializefromobject_index2++) {
/* 247 */               if (serializefromobject_values.isNullAt(serializefromobject_index2)) {
/* 248 */                 serializefromobject_arrayWriter1.setNullInt(serializefromobject_index2);
/* 249 */               } else {
/* 250 */                 final int serializefromobject_element1 = serializefromobject_values.getInt(serializefromobject_index2);
/* 251 */                 serializefromobject_arrayWriter1.write(serializefromobject_index2, serializefromobject_element1);
/* 252 */               }
/* 253 */             }
/* 254 */           }
/* 255 */
/* 256 */         }
/* 257 */
/* 258 */         serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
/* 259 */       }
/* 260 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
/* 261 */       append(serializefromobject_result);
/* 262 */       if (shouldStop()) return;
/* 263 */     }
/* 264 */   }
/* 265 */ }
```

## How was this patch tested?

```
build/mvn -DskipTests clean package && dev/run-tests
```

Additionally in Spark shell:

```
scala> Seq(collection.mutable.HashMap(1 -> 2, 2 -> 3)).toDS().map(_ += (3 -> 4)).collect()
res0: Array[scala.collection.mutable.HashMap[Int,Int]] = Array(Map(2 -> 3, 1 -> 2, 3 -> 4))
```

Author: Michal Senkyr <mike.senkyr@gmail.com>
Author: Michal Šenkýř <mike.senkyr@gmail.com>

Closes #16986 from michalsenkyr/dataset-map-builder.
2017-06-12 08:47:01 +08:00
Zhenhua Wang a7c61c100b [SPARK-21031][SQL] Add alterTableStats to store spark's stats and let alterTable keep existing stats
## What changes were proposed in this pull request?

Currently, hive's stats are read into `CatalogStatistics`, while spark's stats are also persisted through `CatalogStatistics`. As a result, hive's stats can be unexpectedly propagated into spark' stats.

For example, for a catalog table, we read stats from hive, e.g. "totalSize" and put it into `CatalogStatistics`. Then, by using "ALTER TABLE" command, we will store the stats in `CatalogStatistics` into metastore as spark's stats (because we don't know whether it's from spark or not). But spark's stats should be only generated by "ANALYZE" command. This is unexpected from this command.

Secondly, now that we have spark's stats in metastore, after inserting new data, although hive updated "totalSize" in metastore, we still cannot get the right `sizeInBytes` in `CatalogStatistics`, because we respect spark's stats (should not exist) over hive's stats.

A running example is shown in [JIRA](https://issues.apache.org/jira/browse/SPARK-21031).

To fix this, we add a new method `alterTableStats` to store spark's stats, and let `alterTable` keep existing stats.

## How was this patch tested?

Added new tests.

Author: Zhenhua Wang <wzh_zju@163.com>

Closes #18248 from wzhfy/separateHiveStats.
2017-06-12 08:23:04 +08:00
liuxian 5301a19a0e [SPARK-20620][TEST] Improve some unit tests for NullExpressionsSuite and TypeCoercionSuite
## What changes were proposed in this pull request?
add more  datatype for some unit tests

## How was this patch tested?
unit tests

Author: liuxian <liu.xian3@zte.com.cn>

Closes #17880 from 10110346/wip_lx_0506.
2017-06-10 10:42:23 -07:00
Xiao Li 8e96acf71c [SPARK-20211][SQL] Fix the Precision and Scale of Decimal Values when the Input is BigDecimal between -1.0 and 1.0
### What changes were proposed in this pull request?
The precision and scale of decimal values are wrong when the input is BigDecimal between -1.0 and 1.0.

The BigDecimal's precision is the digit count starts from the leftmost nonzero digit based on the [JAVA's BigDecimal definition](https://docs.oracle.com/javase/7/docs/api/java/math/BigDecimal.html). However, our Decimal decision follows the database decimal standard, which is the total number of digits, including both to the left and the right of the decimal point. Thus, this PR is to fix the issue by doing the conversion.

Before this PR, the following queries failed:
```SQL
select 1 > 0.0001
select floor(0.0001)
select ceil(0.0001)
```

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18244 from gatorsmile/bigdecimal.
2017-06-10 10:28:14 -07:00
Xiao Li 571635488d [SPARK-20918][SQL] Use FunctionIdentifier as function identifiers in FunctionRegistry
### What changes were proposed in this pull request?
Currently, the unquoted string of a function identifier is being used as the function identifier in the function registry. This could cause the incorrect the behavior when users use `.` in the function names. This PR is to take the `FunctionIdentifier` as the identifier in the function registry.

- Add one new function `createOrReplaceTempFunction` to `FunctionRegistry`
```Scala
final def createOrReplaceTempFunction(name: String, builder: FunctionBuilder): Unit
```

### How was this patch tested?
Add extra test cases to verify the inclusive bug fixes.

Author: Xiao Li <gatorsmile@gmail.com>
Author: gatorsmile <gatorsmile@gmail.com>

Closes #18142 from gatorsmile/fuctionRegistry.
2017-06-09 10:16:30 -07:00
Bogdan Raducanu cb83ca1433 [SPARK-20854][TESTS] Removing duplicate test case
## What changes were proposed in this pull request?

Removed a duplicate case in "SPARK-20854: select hint syntax with expressions"

## How was this patch tested?
Existing tests.

Author: Bogdan Raducanu <bogdan@databricks.com>

Closes #18217 from bogdanrdc/SPARK-20854-2.
2017-06-06 22:51:10 -07:00
Wenchen Fan c92949ac23 [SPARK-20972][SQL] rename HintInfo.isBroadcastable to broadcast
## What changes were proposed in this pull request?

`HintInfo.isBroadcastable` is actually not an accurate name, it's used to force the planner to broadcast a plan no matter what the data size is, via the hint mechanism. I think `forceBroadcast` is a better name.

And `isBroadcastable` only have 2 possible values: `Some(true)` and `None`, so we can just use boolean type for it.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18189 from cloud-fan/stats.
2017-06-06 22:50:06 -07:00
Bogdan Raducanu 2134196a9c [SPARK-20854][SQL] Extend hint syntax to support expressions
## What changes were proposed in this pull request?

SQL hint syntax:
* support expressions such as strings, numbers, etc. instead of only identifiers as it is currently.
* support multiple hints, which was missing compared to the DataFrame syntax.

DataFrame API:
* support any parameters in DataFrame.hint instead of just strings

## How was this patch tested?
Existing tests. New tests in PlanParserSuite. New suite DataFrameHintSuite.

Author: Bogdan Raducanu <bogdan@databricks.com>

Closes #18086 from bogdanrdc/SPARK-20854.
2017-06-01 15:50:40 -07:00
Yuming Wang 6d05c1c1da [SPARK-20910][SQL] Add build-in SQL function - UUID
## What changes were proposed in this pull request?

Add build-int SQL function - UUID.

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18136 from wangyum/SPARK-20910.
2017-06-01 16:15:24 +09:00
Wenchen Fan 1f5dddffa3 Revert "[SPARK-20392][SQL] Set barrier to prevent re-entering a tree"
This reverts commit 8ce0d8ffb6.
2017-05-30 21:14:55 -07:00
Liang-Chi Hsieh 35b644bd03 [SPARK-20916][SQL] Improve error message for unaliased subqueries in FROM clause
## What changes were proposed in this pull request?

We changed the parser to reject unaliased subqueries in the FROM clause in SPARK-20690. However, the error message that we now give isn't very helpful:

    scala> sql("""SELECT x FROM (SELECT 1 AS x)""")
    org.apache.spark.sql.catalyst.parser.ParseException:
    mismatched input 'FROM' expecting {<EOF>, 'WHERE', 'GROUP', 'ORDER', 'HAVING', 'LIMIT', 'LATERAL', 'WINDOW', 'UNION', 'EXCEPT', 'MINUS', 'INTERSECT', 'SORT', 'CLUSTER', 'DISTRIBUTE'}(line 1, pos 9)

We should modify the parser to throw a more clear error for such queries:

    scala> sql("""SELECT x FROM (SELECT 1 AS x)""")
    org.apache.spark.sql.catalyst.parser.ParseException:
    The unaliased subqueries in the FROM clause are not supported.(line 1, pos 14)

## How was this patch tested?

Modified existing tests to reflect this change.

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

Closes #18141 from viirya/SPARK-20916.
2017-05-30 06:28:43 -07:00
Yuming Wang d797ed0ef1 [SPARK-20909][SQL] Add build-int SQL function - DAYOFWEEK
## What changes were proposed in this pull request?

Add build-int SQL function - DAYOFWEEK

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18134 from wangyum/SPARK-20909.
2017-05-30 15:40:50 +09:00
Kazuaki Ishizaki ef9fd920c3 [SPARK-20750][SQL] Built-in SQL Function Support - REPLACE
## What changes were proposed in this pull request?

This PR adds built-in SQL function `(REPLACE(<string_expression>, <search_string> [, <replacement_string>])`

`REPLACE()` return that string that is replaced all occurrences with given string.

## How was this patch tested?

added new test suites

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

Closes #18047 from kiszk/SPARK-20750.
2017-05-29 11:47:31 -07:00
Tejas Patil f9b59abeae [SPARK-20758][SQL] Add Constant propagation optimization
## What changes were proposed in this pull request?

See class doc of `ConstantPropagation` for the approach used.

## How was this patch tested?

- Added unit tests

Author: Tejas Patil <tejasp@fb.com>

Closes #17993 from tejasapatil/SPARK-20758_const_propagation.
2017-05-29 12:21:34 +02:00
Takeshi Yamamuro 24d34281d7 [SPARK-20841][SQL] Support table column aliases in FROM clause
## What changes were proposed in this pull request?
This pr added parsing rules to support table column aliases in FROM clause.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #18079 from maropu/SPARK-20841.
2017-05-28 13:23:18 -07:00
Xiao Li 06c155c90d [SPARK-20908][SQL] Cache Manager: Hint should be ignored in plan matching
### What changes were proposed in this pull request?

In Cache manager, the plan matching should ignore Hint.
```Scala
      val df1 = spark.range(10).join(broadcast(spark.range(10)))
      df1.cache()
      spark.range(10).join(spark.range(10)).explain()
```
The output plan of the above query shows that the second query is  not using the cached data of the first query.
```
BroadcastNestedLoopJoin BuildRight, Inner
:- *Range (0, 10, step=1, splits=2)
+- BroadcastExchange IdentityBroadcastMode
   +- *Range (0, 10, step=1, splits=2)
```

After the fix, the plan becomes
```
InMemoryTableScan [id#20L, id#23L]
   +- InMemoryRelation [id#20L, id#23L], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)
         +- BroadcastNestedLoopJoin BuildRight, Inner
            :- *Range (0, 10, step=1, splits=2)
            +- BroadcastExchange IdentityBroadcastMode
               +- *Range (0, 10, step=1, splits=2)
```

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18131 from gatorsmile/HintCache.
2017-05-27 21:32:18 -07:00
liuxian 3969a8078e [SPARK-20876][SQL] If the input parameter is float type for ceil or floor,the result is not we expected
## What changes were proposed in this pull request?

spark-sql>SELECT ceil(cast(12345.1233 as float));
spark-sql>12345
For this case, the result we expected is `12346`
spark-sql>SELECT floor(cast(-12345.1233 as float));
spark-sql>-12345
For this case, the result we expected is `-12346`

Because in `Ceil` or `Floor`, `inputTypes` has no FloatType, so it is converted to LongType.
## How was this patch tested?

After the modification:
spark-sql>SELECT ceil(cast(12345.1233 as float));
spark-sql>12346
spark-sql>SELECT floor(cast(-12345.1233 as float));
spark-sql>-12346

Author: liuxian <liu.xian3@zte.com.cn>

Closes #18103 from 10110346/wip-lx-0525-1.
2017-05-27 16:23:45 -07:00
Yuming Wang a0f8a072e3 [SPARK-20748][SQL] Add built-in SQL function CH[A]R.
## What changes were proposed in this pull request?
Add built-in SQL function `CH[A]R`:
For `CHR(bigint|double n)`, returns the ASCII character having the binary equivalent to `n`. If n is larger than 256 the result is equivalent to CHR(n % 256)

## How was this patch tested?
unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18019 from wangyum/SPARK-20748.
2017-05-26 20:59:14 -07:00
Liang-Chi Hsieh 8ce0d8ffb6 [SPARK-20392][SQL] Set barrier to prevent re-entering a tree
## What changes were proposed in this pull request?

It is reported that there is performance downgrade when applying ML pipeline for dataset with many columns but few rows.

A big part of the performance downgrade comes from some operations (e.g., `select`) on DataFrame/Dataset which re-create new DataFrame/Dataset with a new `LogicalPlan`. The cost can be ignored in the usage of SQL, normally.

However, it's not rare to chain dozens of pipeline stages in ML. When the query plan grows incrementally during running those stages, the total cost spent on re-creation of DataFrame grows too. In particular, the `Analyzer` will go through the big query plan even most part of it is analyzed.

By eliminating part of the cost, the time to run the example code locally is reduced from about 1min to about 30 secs.

In particular, the time applying the pipeline locally is mostly spent on calling transform of the 137 `Bucketizer`s. Before the change, each call of `Bucketizer`'s transform can cost about 0.4 sec. So the total time spent on all `Bucketizer`s' transform is about 50 secs. After the change, each call only costs about 0.1 sec.

<del>We also make `boundEnc` as lazy variable to reduce unnecessary running time.</del>

### Performance improvement

The codes and datasets provided by Barry Becker to re-produce this issue and benchmark can be found on the JIRA.

Before this patch: about 1 min
After this patch: about 20 secs

## How was this patch tested?

Existing tests.

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

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

Closes #17770 from viirya/SPARK-20392.
2017-05-26 13:45:55 +08:00
Reynold Xin a64746677b [SPARK-20867][SQL] Move hints from Statistics into HintInfo class
## What changes were proposed in this pull request?
This is a follow-up to SPARK-20857 to move the broadcast hint from Statistics into a new HintInfo class, so we can be more flexible in adding new hints in the future.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #18087 from rxin/SPARK-20867.
2017-05-24 13:57:19 -07:00
Reynold Xin 0d589ba00b [SPARK-20857][SQL] Generic resolved hint node
## What changes were proposed in this pull request?
This patch renames BroadcastHint to ResolvedHint (and Hint to UnresolvedHint) so the hint framework is more generic and would allow us to introduce other hint types in the future without introducing new hint nodes.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #18072 from rxin/SPARK-20857.
2017-05-23 18:44:49 +02:00
Liang-Chi Hsieh 442287ae29 [SPARK-20399][SQL][FOLLOW-UP] Add a config to fallback string literal parsing consistent with old sql parser behavior
## What changes were proposed in this pull request?

As srowen pointed in 609ba5f2b9 (commitcomment-22221259), the previous tests are not proper.

This follow-up is going to fix the tests.

## How was this patch tested?

Jenkins tests.

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

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

Closes #18048 from viirya/SPARK-20399-follow-up.
2017-05-23 16:09:38 +08:00
Xiao Li a2460be9c3 [SPARK-17410][SPARK-17284] Move Hive-generated Stats Info to HiveClientImpl
### What changes were proposed in this pull request?

After we adding a new field `stats` into `CatalogTable`, we should not expose Hive-specific Stats metadata to `MetastoreRelation`. It complicates all the related codes. It also introduces a bug in `SHOW CREATE TABLE`. The statistics-related table properties should be skipped by `SHOW CREATE TABLE`, since it could be incorrect in the newly created table. See the Hive JIRA: https://issues.apache.org/jira/browse/HIVE-13792

Also fix the issue to fill Hive-generated RowCounts to our stats.

This PR is to handle Hive-specific Stats metadata in `HiveClientImpl`.
### How was this patch tested?

Added a few test cases.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #14971 from gatorsmile/showCreateTableNew.
2017-05-22 17:28:30 -07:00
Yuming Wang 9b09101938 [SPARK-20751][SQL][FOLLOWUP] Add cot test in MathExpressionsSuite
## What changes were proposed in this pull request?

Add cot test in MathExpressionsSuite as https://github.com/apache/spark/pull/17999#issuecomment-302832794.

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18039 from wangyum/SPARK-20751-test.
2017-05-22 13:05:05 -07:00
gatorsmile f3ed62a381 [SPARK-20831][SQL] Fix INSERT OVERWRITE data source tables with IF NOT EXISTS
### What changes were proposed in this pull request?
Currently, we have a bug when we specify `IF NOT EXISTS` in `INSERT OVERWRITE` data source tables. For example, given a query:
```SQL
INSERT OVERWRITE TABLE $tableName partition (b=2, c=3) IF NOT EXISTS SELECT 9, 10
```
we will get the following error:
```
unresolved operator 'InsertIntoTable Relation[a#425,d#426,b#427,c#428] parquet, Map(b -> Some(2), c -> Some(3)), true, true;;
'InsertIntoTable Relation[a#425,d#426,b#427,c#428] parquet, Map(b -> Some(2), c -> Some(3)), true, true
+- Project [cast(9#423 as int) AS a#429, cast(10#424 as int) AS d#430]
   +- Project [9 AS 9#423, 10 AS 10#424]
      +- OneRowRelation$
```

This PR is to fix the issue to follow the behavior of Hive serde tables
> INSERT OVERWRITE will overwrite any existing data in the table or partition unless IF NOT EXISTS is provided for a partition

### How was this patch tested?
Modified an existing test case

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18050 from gatorsmile/insertPartitionIfNotExists.
2017-05-22 22:24:50 +08:00
caoxuewen 3c9eef35a8 [SPARK-20786][SQL] Improve ceil and floor handle the value which is not expected
## What changes were proposed in this pull request?

spark-sql>SELECT ceil(1234567890123456);
1234567890123456

spark-sql>SELECT ceil(12345678901234567);
12345678901234568

spark-sql>SELECT ceil(123456789012345678);
123456789012345680

when the length of the getText is greater than 16. long to double will be precision loss.

but mysql handle the value is ok.

mysql> SELECT ceil(1234567890123456);
+------------------------+
| ceil(1234567890123456) |
+------------------------+
|       1234567890123456 |
+------------------------+
1 row in set (0.00 sec)

mysql> SELECT ceil(12345678901234567);
+-------------------------+
| ceil(12345678901234567) |
+-------------------------+
|       12345678901234567 |
+-------------------------+
1 row in set (0.00 sec)

mysql> SELECT ceil(123456789012345678);
+--------------------------+
| ceil(123456789012345678) |
+--------------------------+
|       123456789012345678 |
+--------------------------+
1 row in set (0.00 sec)

## How was this patch tested?

Supplement the unit test.

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

Closes #18016 from heary-cao/ceil_long.
2017-05-21 22:39:07 -07:00
liuxian ea3b1e352a [SPARK-20763][SQL] The function of month and day return the value which is not we expected.
## What changes were proposed in this pull request?
spark-sql>select month("1582-09-28");
spark-sql>10
For this case, the expected result is 9, but it is 10.

spark-sql>select day("1582-04-18");
spark-sql>28
For this case, the expected result is 18, but it is 28.

when the date  before "1582-10-04", the function of `month` and `day` return the value which is not we expected.

## How was this patch tested?
unit tests

Author: liuxian <liu.xian3@zte.com.cn>

Closes #17997 from 10110346/wip_lx_0516.
2017-05-19 10:25:21 -07:00
Ala Luszczak ce8edb8bf4 [SPARK-20798] GenerateUnsafeProjection should check if a value is null before calling the getter
## What changes were proposed in this pull request?

GenerateUnsafeProjection.writeStructToBuffer() did not honor the assumption that the caller must make sure that a value is not null before using the getter. This could lead to various errors. This change fixes that behavior.

Example of code generated before:
```scala
/* 059 */         final UTF8String fieldName = value.getUTF8String(0);
/* 060 */         if (value.isNullAt(0)) {
/* 061 */           rowWriter1.setNullAt(0);
/* 062 */         } else {
/* 063 */           rowWriter1.write(0, fieldName);
/* 064 */         }
```

Example of code generated now:
```scala
/* 060 */         boolean isNull1 = value.isNullAt(0);
/* 061 */         UTF8String value1 = isNull1 ? null : value.getUTF8String(0);
/* 062 */         if (isNull1) {
/* 063 */           rowWriter1.setNullAt(0);
/* 064 */         } else {
/* 065 */           rowWriter1.write(0, value1);
/* 066 */         }
```

## How was this patch tested?

Adds GenerateUnsafeProjectionSuite.

Author: Ala Luszczak <ala@databricks.com>

Closes #18030 from ala/fix-generate-unsafe-projection.
2017-05-19 13:18:48 +02:00
Xingbo Jiang b7aac15d56 [SPARK-20700][SQL] InferFiltersFromConstraints stackoverflows for query (v2)
## What changes were proposed in this pull request?

In the previous approach we used `aliasMap` to link an `Attribute` to the expression with potentially the form `f(a, b)`, but we only searched the `expressions` and `children.expressions` for this, which is not enough when an `Alias` may lies deep in the logical plan. In that case, we can't generate the valid equivalent constraint classes and thus we fail at preventing the recursive deductions.

We fix this problem by collecting all `Alias`s from the logical plan.

## How was this patch tested?

No additional test case is added, but do modified one test case to cover this situation.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #18020 from jiangxb1987/inferConstrants.
2017-05-17 23:32:31 -07:00
Liang-Chi Hsieh 7463a88be6 [SPARK-20690][SQL] Subqueries in FROM should have alias names
## What changes were proposed in this pull request?

We add missing attributes into Filter in Analyzer. But we shouldn't do it through subqueries like this:

    select 1 from  (select 1 from onerow t1 LIMIT 1) where  t1.c1=1

This query works in current codebase. However, the outside where clause shouldn't be able to refer `t1.c1` attribute.

The root cause is we allow subqueries in FROM have no alias names previously, it is confusing and isn't supported by various databases such as MySQL, Postgres, Oracle. We shouldn't support it too.

## How was this patch tested?

Jenkins tests.

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

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

Closes #17935 from viirya/SPARK-20690.
2017-05-17 12:57:35 +08:00
Tejas Patil d2416925c4 [SPARK-17729][SQL] Enable creating hive bucketed tables
## What changes were proposed in this pull request?

Hive allows inserting data to bucketed table without guaranteeing bucketed and sorted-ness based on these two configs : `hive.enforce.bucketing` and `hive.enforce.sorting`.

What does this PR achieve ?
- Spark will disallow users from writing outputs to hive bucketed tables by default (given that output won't adhere with Hive's semantics).
- IF user still wants to write to hive bucketed table, the only resort is to use `hive.enforce.bucketing=false` and `hive.enforce.sorting=false` which means user does NOT care about bucketing guarantees.

Changes done in this PR:
- Extract table's bucketing information in `HiveClientImpl`
- While writing table info to metastore, `HiveClientImpl` now populates the bucketing information in the hive `Table` object
- `InsertIntoHiveTable` allows inserts to bucketed table only if both `hive.enforce.bucketing` and `hive.enforce.sorting` are `false`

Ability to create bucketed tables will enable adding test cases to Spark while I add more changes related to hive bucketing support. Design doc for hive hive bucketing support : https://docs.google.com/document/d/1a8IDh23RAkrkg9YYAeO51F4aGO8-xAlupKwdshve2fc/edit#

## How was this patch tested?
- Added test for creating bucketed and sorted table.
- Added test to ensure that INSERTs fail if strict bucket / sort is enforced
- Added test to ensure that INSERTs can go through if strict bucket / sort is NOT enforced
- Added test to validate that bucketing information shows up in output of DESC FORMATTED
- Added test to ensure that `SHOW CREATE TABLE` works for hive bucketed tables

Author: Tejas Patil <tejasp@fb.com>

Closes #17644 from tejasapatil/SPARK-17729_create_bucketed_table.
2017-05-16 01:47:23 +08:00
Takeshi Yamamuro b0888d1ac3 [SPARK-20730][SQL] Add an optimizer rule to combine nested Concat
## What changes were proposed in this pull request?
This pr added a new Optimizer rule to combine nested Concat. The master supports a pipeline operator '||' to concatenate strings in #17711 (This pr is follow-up). Since the parser currently generates nested Concat expressions, the optimizer needs to combine the nested expressions.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17970 from maropu/SPARK-20730.
2017-05-15 16:24:55 +08:00
hyukjinkwon 720708ccdd [SPARK-20639][SQL] Add single argument support for to_timestamp in SQL with documentation improvement
## What changes were proposed in this pull request?

This PR proposes three things as below:

- Use casting rules to a timestamp in `to_timestamp` by default (it was `yyyy-MM-dd HH:mm:ss`).

- Support single argument for `to_timestamp` similarly with APIs in other languages.

  For example, the one below works

  ```
  import org.apache.spark.sql.functions._
  Seq("2016-12-31 00:12:00.00").toDF("a").select(to_timestamp(col("a"))).show()
  ```

  prints

  ```
  +----------------------------------------+
  |to_timestamp(`a`, 'yyyy-MM-dd HH:mm:ss')|
  +----------------------------------------+
  |                     2016-12-31 00:12:00|
  +----------------------------------------+
  ```

  whereas this does not work in SQL.

  **Before**

  ```
  spark-sql> SELECT to_timestamp('2016-12-31 00:12:00');
  Error in query: Invalid number of arguments for function to_timestamp; line 1 pos 7
  ```

  **After**

  ```
  spark-sql> SELECT to_timestamp('2016-12-31 00:12:00');
  2016-12-31 00:12:00
  ```

- Related document improvement for SQL function descriptions and other API descriptions accordingly.

  **Before**

  ```
  spark-sql> DESCRIBE FUNCTION extended to_date;
  ...
  Usage: to_date(date_str, fmt) - Parses the `left` expression with the `fmt` expression. Returns null with invalid input.
  Extended Usage:
      Examples:
        > SELECT to_date('2016-12-31', 'yyyy-MM-dd');
         2016-12-31
  ```

  ```
  spark-sql> DESCRIBE FUNCTION extended to_timestamp;
  ...
  Usage: to_timestamp(timestamp, fmt) - Parses the `left` expression with the `format` expression to a timestamp. Returns null with invalid input.
  Extended Usage:
      Examples:
        > SELECT to_timestamp('2016-12-31', 'yyyy-MM-dd');
         2016-12-31 00:00:00.0
  ```

  **After**

  ```
  spark-sql> DESCRIBE FUNCTION extended to_date;
  ...
  Usage:
      to_date(date_str[, fmt]) - Parses the `date_str` expression with the `fmt` expression to
        a date. Returns null with invalid input. By default, it follows casting rules to a date if
        the `fmt` is omitted.

  Extended Usage:
      Examples:
        > SELECT to_date('2009-07-30 04:17:52');
         2009-07-30
        > SELECT to_date('2016-12-31', 'yyyy-MM-dd');
         2016-12-31
  ```

  ```
  spark-sql> DESCRIBE FUNCTION extended to_timestamp;
  ...
   Usage:
      to_timestamp(timestamp[, fmt]) - Parses the `timestamp` expression with the `fmt` expression to
        a timestamp. Returns null with invalid input. By default, it follows casting rules to
        a timestamp if the `fmt` is omitted.

  Extended Usage:
      Examples:
        > SELECT to_timestamp('2016-12-31 00:12:00');
         2016-12-31 00:12:00
        > SELECT to_timestamp('2016-12-31', 'yyyy-MM-dd');
         2016-12-31 00:00:00
  ```

## How was this patch tested?

Added tests in `datetime.sql`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17901 from HyukjinKwon/to_timestamp_arg.
2017-05-12 16:42:58 +08:00
liuxian 2b36eb696f [SPARK-20665][SQL] Bround" and "Round" function return NULL
## What changes were proposed in this pull request?
   spark-sql>select bround(12.3, 2);
   spark-sql>NULL
For this case,  the expected result is 12.3, but it is null.
So ,when the second parameter is bigger than "decimal.scala", the result is not we expected.
"round" function  has the same problem. This PR can solve the problem for both of them.

## How was this patch tested?
unit test cases in MathExpressionsSuite and MathFunctionsSuite

Author: liuxian <liu.xian3@zte.com.cn>

Closes #17906 from 10110346/wip_lx_0509.
2017-05-12 11:38:50 +08:00
Liang-Chi Hsieh 609ba5f2b9 [SPARK-20399][SQL] Add a config to fallback string literal parsing consistent with old sql parser behavior
## What changes were proposed in this pull request?

The new SQL parser is introduced into Spark 2.0. All string literals are unescaped in parser. Seems it bring an issue regarding the regex pattern string.

The following codes can reproduce it:

    val data = Seq("\u0020\u0021\u0023", "abc")
    val df = data.toDF()

    // 1st usage: works in 1.6
    // Let parser parse pattern string
    val rlike1 = df.filter("value rlike '^\\x20[\\x20-\\x23]+$'")
    // 2nd usage: works in 1.6, 2.x
    // Call Column.rlike so the pattern string is a literal which doesn't go through parser
    val rlike2 = df.filter($"value".rlike("^\\x20[\\x20-\\x23]+$"))

    // In 2.x, we need add backslashes to make regex pattern parsed correctly
    val rlike3 = df.filter("value rlike '^\\\\x20[\\\\x20-\\\\x23]+$'")

Follow the discussion in #17736, this patch adds a config to fallback to 1.6 string literal parsing and mitigate migration issue.

## How was this patch tested?

Jenkins tests.

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

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

Closes #17887 from viirya/add-config-fallback-string-parsing.
2017-05-12 11:15:10 +08:00
Takeshi Yamamuro 8c67aa7f00 [SPARK-20311][SQL] Support aliases for table value functions
## What changes were proposed in this pull request?
This pr added parsing rules to support aliases in table value functions.
The previous pr (#17666) has been reverted because of the regression. This new pr fixed the regression and add tests in `SQLQueryTestSuite`.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17928 from maropu/SPARK-20311-3.
2017-05-11 18:09:31 +08:00
wangzhenhua 76e4a5566b [SPARK-20678][SQL] Ndv for columns not in filter condition should also be updated
## What changes were proposed in this pull request?

In filter estimation, we update column stats for those columns in filter condition. However, if the number of rows decreases after the filter (i.e. the overall selectivity is less than 1), we need to update (scale down) the number of distinct values (NDV) for all columns, no matter they are in filter conditions or not.

This pr also fixes the inconsistency of rounding mode for ndv and rowCount.

## How was this patch tested?

Added new tests.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17918 from wzhfy/scaleDownNdvAfterFilter.
2017-05-10 19:42:49 +08:00
Josh Rosen a90c5cd822 [SPARK-20686][SQL] PropagateEmptyRelation incorrectly handles aggregate without grouping
## What changes were proposed in this pull request?

The query

```
SELECT 1 FROM (SELECT COUNT(*) WHERE FALSE) t1
```

should return a single row of output because the subquery is an aggregate without a group-by and thus should return a single row. However, Spark incorrectly returns zero rows.

This is caused by SPARK-16208 / #13906, a patch which added an optimizer rule to propagate EmptyRelation through operators. The logic for handling aggregates is wrong: it checks whether aggregate expressions are non-empty for deciding whether the output should be empty, whereas it should be checking grouping expressions instead:

An aggregate with non-empty grouping expression will return one output row per group. If the input to the grouped aggregate is empty then all groups will be empty and thus the output will be empty. It doesn't matter whether the aggregation output columns include aggregate expressions since that won't affect the number of output rows.

If the grouping expressions are empty, however, then the aggregate will always produce a single output row and thus we cannot propagate the EmptyRelation.

The current implementation is incorrect and also misses an optimization opportunity by not propagating EmptyRelation in the case where a grouped aggregate has aggregate expressions (in other words, `SELECT COUNT(*) from emptyRelation GROUP BY x` would _not_ be optimized to `EmptyRelation` in the old code, even though it safely could be).

This patch resolves this issue by modifying `PropagateEmptyRelation` to consider only the presence/absence of grouping expressions, not the aggregate functions themselves, when deciding whether to propagate EmptyRelation.

## How was this patch tested?

- Added end-to-end regression tests in `SQLQueryTest`'s `group-by.sql` file.
- Updated unit tests in `PropagateEmptyRelationSuite`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #17929 from JoshRosen/fix-PropagateEmptyRelation.
2017-05-10 14:36:36 +08:00
Yin Huai f79aa285cf Revert "[SPARK-20311][SQL] Support aliases for table value functions"
This reverts commit 714811d0b5.
2017-05-09 14:47:45 -07:00
Takeshi Yamamuro 714811d0b5 [SPARK-20311][SQL] Support aliases for table value functions
## What changes were proposed in this pull request?
This pr added parsing rules to support aliases in table value functions.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17666 from maropu/SPARK-20311.
2017-05-09 20:22:51 +08:00
Sean Owen 16fab6b0ef [SPARK-20523][BUILD] Clean up build warnings for 2.2.0 release
## What changes were proposed in this pull request?

Fix build warnings primarily related to Breeze 0.13 operator changes, Java style problems

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #17803 from srowen/SPARK-20523.
2017-05-03 10:18:35 +01:00
Burak Yavuz 86174ea89b [SPARK-20549] java.io.CharConversionException: Invalid UTF-32' in JsonToStructs
## What changes were proposed in this pull request?

A fix for the same problem was made in #17693 but ignored `JsonToStructs`. This PR uses the same fix for `JsonToStructs`.

## How was this patch tested?

Regression test

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #17826 from brkyvz/SPARK-20549.
2017-05-02 14:08:16 +08:00
ptkool 259860d23d [SPARK-20463] Add support for IS [NOT] DISTINCT FROM.
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

Tested using unit tests, integration tests, manual tests.

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

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

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

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

**Before**

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

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

**After**

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

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

## How was this patch tested?

Unit tests in `DataTypeParserSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17784 from HyukjinKwon/SPARK-20492.
2017-04-30 08:24:10 -07:00
Kris Mok 26ac2ce05c [SPARK-20482][SQL] Resolving Casts is too strict on having time zone set
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

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

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

Closes #17777 from rednaxelafx/fix-catalyst-cast-timezone.
2017-04-27 12:08:16 -07:00
Eric Wasserman 57e1da3946 [SPARK-16548][SQL] Inconsistent error handling in JSON parsing SQL functions
## What changes were proposed in this pull request?

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

    public JsonParser createParser(String content)

## How was this patch tested?

existing unit tests

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

Author: Eric Wasserman <ericw@sgn.com>

Closes #17693 from ewasserman/SPARK-20314.
2017-04-26 11:42:43 +08:00
Kazuaki Ishizaki a750a59597 [SPARK-20341][SQL] Support BigInt's value that does not fit in long value range
## What changes were proposed in this pull request?

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

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

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

## How was this patch tested?

Add new test suite into `DecimalSuite`

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

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

The following events are fired per object:

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

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

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

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

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

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

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

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

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

Closes #17641 from hvanhovell/SPARK-20329.
2017-04-21 10:06:12 +08:00
ptkool 63824b2c8e [SPARK-20350] Add optimization rules to apply Complementation Laws.
## What changes were proposed in this pull request?

Apply Complementation Laws during boolean expression simplification.

## How was this patch tested?

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

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

Closes #17650 from ptkool/apply_complementation_laws.
2017-04-20 09:51:13 +08:00
Kazuaki Ishizaki e468a96c40 [SPARK-20254][SQL] Remove unnecessary data conversion for Dataset with primitive array
## What changes were proposed in this pull request?

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

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

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

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

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

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

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

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

## How was this patch tested?

Add test suites into `DatasetPrimitiveSuite`

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

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

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

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

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

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

## How was this patch tested?

Add a corresponding test case.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17668 from wzhfy/recursiveReorder.
2017-04-18 20:12:21 +08:00
Jacek Laskowski 33ea908af9 [TEST][MINOR] Replace repartitionBy with distribute in CollapseRepartitionSuite
## What changes were proposed in this pull request?

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

## How was this patch tested?

local build and `catalyst/testOnly *CollapseRepartitionSuite`

Author: Jacek Laskowski <jacek@japila.pl>

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

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

---

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

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

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

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

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

Extra case in regex unit tests.

Author: Jakob Odersky <jakob@odersky.com>

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

Closes #15398 from jodersky/SPARK-17647.
2017-04-17 11:17:57 -07:00
wangzhenhua fb036c4413 [SPARK-20318][SQL] Use Catalyst type for min/max in ColumnStat for ease of estimation
## What changes were proposed in this pull request?

Currently when estimating predicates like col > literal or col = literal, we will update min or max in column stats based on literal value. However, literal value is of Catalyst type (internal type), while min/max is of external type. Then for the next predicate, we again need to do type conversion to compare and update column stats. This is awkward and causes many unnecessary conversions in estimation.

To solve this, we use Catalyst type for min/max in `ColumnStat`. Note that the persistent format in metastore is still of external type, so there's no inconsistency for statistics in metastore.

This pr also fixes a bug for boolean type in `IN` condition.

## How was this patch tested?

The changes for ColumnStat are covered by existing tests.
For bug fix, a new test for boolean type in IN condition is added

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17630 from wzhfy/refactorColumnStat.
2017-04-14 19:16:47 +08:00
Ioana Delaney fbe4216e1e [SPARK-20233][SQL] Apply star-join filter heuristics to dynamic programming join enumeration
## What changes were proposed in this pull request?

Implements star-join filter to reduce the search space for dynamic programming join enumeration. Consider the following join graph:

```
T1       D1 - T2 - T3
  \     /
    F1
     |
    D2

star-join: {F1, D1, D2}
non-star: {T1, T2, T3}
```
The following join combinations will be generated:
```
level 0: (F1), (D1), (D2), (T1), (T2), (T3)
level 1: {F1, D1}, {F1, D2}, {T2, T3}
level 2: {F1, D1, D2}
level 3: {F1, D1, D2, T1}, {F1, D1, D2, T2}
level 4: {F1, D1, D2, T1, T2}, {F1, D1, D2, T2, T3 }
level 6: {F1, D1, D2, T1, T2, T3}
```

## How was this patch tested?

New test suite ```StarJOinCostBasedReorderSuite.scala```.

Author: Ioana Delaney <ioanamdelaney@gmail.com>

Closes #17546 from ioana-delaney/starSchemaCBOv3.
2017-04-13 22:27:04 +08:00
Xiao Li 504e62e2f4 [SPARK-20303][SQL] Rename createTempFunction to registerFunction
### What changes were proposed in this pull request?
Session catalog API `createTempFunction` is being used by Hive build-in functions, persistent functions, and temporary functions. Thus, the name is confusing. This PR is to rename it by `registerFunction`. Also we can move construction of `FunctionBuilder` and `ExpressionInfo` into the new `registerFunction`, instead of duplicating the logics everywhere.

In the next PRs, the remaining Function-related APIs also need cleanups.

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17615 from gatorsmile/cleanupCreateTempFunction.
2017-04-12 09:01:26 -07:00
hyukjinkwon ceaf77ae43 [SPARK-18692][BUILD][DOCS] Test Java 8 unidoc build on Jenkins
## What changes were proposed in this pull request?

This PR proposes to run Spark unidoc to test Javadoc 8 build as Javadoc 8 is easily re-breakable.

There are several problems with it:

- It introduces little extra bit of time to run the tests. In my case, it took 1.5 mins more (`Elapsed :[94.8746569157]`). How it was tested is described in "How was this patch tested?".

- > One problem that I noticed was that Unidoc appeared to be processing test sources: if we can find a way to exclude those from being processed in the first place then that might significantly speed things up.

  (see  joshrosen's [comment](https://issues.apache.org/jira/browse/SPARK-18692?focusedCommentId=15947627&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15947627))

To complete this automated build, It also suggests to fix existing Javadoc breaks / ones introduced by test codes as described above.

There fixes are similar instances that previously fixed. Please refer https://github.com/apache/spark/pull/15999 and https://github.com/apache/spark/pull/16013

Note that this only fixes **errors** not **warnings**. Please see my observation https://github.com/apache/spark/pull/17389#issuecomment-288438704 for spurious errors by warnings.

## How was this patch tested?

Manually via `jekyll build` for building tests. Also, tested via running `./dev/run-tests`.

This was tested via manually adding `time.time()` as below:

```diff
     profiles_and_goals = build_profiles + sbt_goals

     print("[info] Building Spark unidoc (w/Hive 1.2.1) using SBT with these arguments: ",
           " ".join(profiles_and_goals))

+    import time
+    st = time.time()
     exec_sbt(profiles_and_goals)
+    print("Elapsed :[%s]" % str(time.time() - st))
```

produces

```
...
========================================================================
Building Unidoc API Documentation
========================================================================
...
[info] Main Java API documentation successful.
...
Elapsed :[94.8746569157]
...

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17477 from HyukjinKwon/SPARK-18692.
2017-04-12 12:38:48 +01:00
Reynold Xin ffc57b0118 [SPARK-20302][SQL] Short circuit cast when from and to types are structurally the same
## What changes were proposed in this pull request?
When we perform a cast expression and the from and to types are structurally the same (having the same structure but different field names), we should be able to skip the actual cast.

## How was this patch tested?
Added unit tests for the newly introduced functions.

Author: Reynold Xin <rxin@databricks.com>

Closes #17614 from rxin/SPARK-20302.
2017-04-12 01:30:00 -07:00
DB Tsai 8ad63ee158 [SPARK-20291][SQL] NaNvl(FloatType, NullType) should not be cast to NaNvl(DoubleType, DoubleType)
## What changes were proposed in this pull request?

`NaNvl(float value, null)` will be converted into `NaNvl(float value, Cast(null, DoubleType))` and finally `NaNvl(Cast(float value, DoubleType), Cast(null, DoubleType))`.

This will cause mismatching in the output type when the input type is float.

By adding extra rule in TypeCoercion can resolve this issue.

## How was this patch tested?

unite tests.

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

Author: DB Tsai <dbt@netflix.com>

Closes #17606 from dbtsai/fixNaNvl.
2017-04-12 11:19:20 +08:00
Wenchen Fan c8706980ae [SPARK-20274][SQL] support compatible array element type in encoder
## What changes were proposed in this pull request?

This is a regression caused by SPARK-19716.

Before SPARK-19716, we will cast an array field to the expected array type. However, after SPARK-19716, the cast is removed, but we forgot to push the cast to the element level.

## How was this patch tested?

new regression tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17587 from cloud-fan/array.
2017-04-11 20:21:04 +08:00
Sean Owen a26e3ed5e4 [SPARK-20156][CORE][SQL][STREAMING][MLLIB] Java String toLowerCase "Turkish locale bug" causes Spark problems
## What changes were proposed in this pull request?

Add Locale.ROOT to internal calls to String `toLowerCase`, `toUpperCase`, to avoid inadvertent locale-sensitive variation in behavior (aka the "Turkish locale problem").

The change looks large but it is just adding `Locale.ROOT` (the locale with no country or language specified) to every call to these methods.

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #17527 from srowen/SPARK-20156.
2017-04-10 20:11:56 +01:00
Xiao Li fd711ea13e [SPARK-20273][SQL] Disallow Non-deterministic Filter push-down into Join Conditions
## What changes were proposed in this pull request?
```
sql("SELECT t1.b, rand(0) as r FROM cachedData, cachedData t1 GROUP BY t1.b having r > 0.5").show()
```
We will get the following error:
```
Job aborted due to stage failure: Task 1 in stage 4.0 failed 1 times, most recent failure: Lost task 1.0 in stage 4.0 (TID 8, localhost, executor driver): java.lang.NullPointerException
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificPredicate.eval(Unknown Source)
	at org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec$$anonfun$org$apache$spark$sql$execution$joins$BroadcastNestedLoopJoinExec$$boundCondition$1.apply(BroadcastNestedLoopJoinExec.scala:87)
	at org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec$$anonfun$org$apache$spark$sql$execution$joins$BroadcastNestedLoopJoinExec$$boundCondition$1.apply(BroadcastNestedLoopJoinExec.scala:87)
	at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:463)
```
Filters could be pushed down to the join conditions by the optimizer rule `PushPredicateThroughJoin`. However, Analyzer [blocks users to add non-deterministics conditions](https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala#L386-L395) (For details, see the PR https://github.com/apache/spark/pull/7535).

We should not push down non-deterministic conditions; otherwise, we need to explicitly initialize the non-deterministic expressions. This PR is to simply block it.

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17585 from gatorsmile/joinRandCondition.
2017-04-10 09:15:04 -07:00
hyukjinkwon 5acaf8c0c6 [SPARK-19518][SQL] IGNORE NULLS in first / last in SQL
## What changes were proposed in this pull request?

This PR proposes to add `IGNORE NULLS` keyword in `first`/`last` in Spark's parser likewise http://docs.oracle.com/cd/B19306_01/server.102/b14200/functions057.htm.  This simply maps the keywords to existing `ignoreNullsExpr`.

**Before**

```scala
scala> sql("select first('a' IGNORE NULLS)").show()
```

```
org.apache.spark.sql.catalyst.parser.ParseException:
extraneous input 'NULLS' expecting {')', ','}(line 1, pos 24)

== SQL ==
select first('a' IGNORE NULLS)
------------------------^^^

  at org.apache.spark.sql.catalyst.parser.ParseException.withCommand(ParseDriver.scala:210)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:112)
  at org.apache.spark.sql.execution.SparkSqlParser.parse(SparkSqlParser.scala:46)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parsePlan(ParseDriver.scala:66)
  at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:622)
  ... 48 elided
```

**After**

```scala
scala> sql("select first('a' IGNORE NULLS)").show()
```

```
+--------------+
|first(a, true)|
+--------------+
|             a|
+--------------+
```

## How was this patch tested?

Unit tests in `ExpressionParserSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17566 from HyukjinKwon/SPARK-19518.
2017-04-10 17:45:27 +02:00
Wenchen Fan 7577e9c356 [SPARK-20246][SQL] should not push predicate down through aggregate with non-deterministic expressions
## What changes were proposed in this pull request?

Similar to `Project`, when `Aggregate` has non-deterministic expressions, we should not push predicate down through it, as it will change the number of input rows and thus change the evaluation result of non-deterministic expressions in `Aggregate`.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17562 from cloud-fan/filter.
2017-04-07 20:54:18 -07:00
Ioana Delaney 4000f128b7 [SPARK-20231][SQL] Refactor star schema code for the subsequent star join detection in CBO
## What changes were proposed in this pull request?

This commit moves star schema code from ```join.scala``` to ```StarSchemaDetection.scala```. It also applies some minor fixes in ```StarJoinReorderSuite.scala```.

## How was this patch tested?
Run existing ```StarJoinReorderSuite.scala```.

Author: Ioana Delaney <ioanamdelaney@gmail.com>

Closes #17544 from ioana-delaney/starSchemaCBOv2.
2017-04-05 18:02:53 -07:00
Wenchen Fan 295747e597 [SPARK-19716][SQL] support by-name resolution for struct type elements in array
## What changes were proposed in this pull request?

Previously when we construct deserializer expression for array type, we will first cast the corresponding field to expected array type and then apply `MapObjects`.

However, by doing that, we lose the opportunity to do by-name resolution for struct type inside array type. In this PR, I introduce a `UnresolvedMapObjects` to hold the lambda function and the input array expression. Then during analysis, after the input array expression is resolved, we get the actual array element type and apply by-name resolution. Then we don't need to add `Cast` for array type when constructing the deserializer expression, as the element type is determined later at analyzer.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17398 from cloud-fan/dataset.
2017-04-04 16:38:32 -07:00
Wenchen Fan 402bf2a50d [SPARK-20204][SQL] remove SimpleCatalystConf and CatalystConf type alias
## What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/17285 .

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17521 from cloud-fan/conf.
2017-04-04 11:56:21 -07:00
Ron Hu e7877fd472 [SPARK-19408][SQL] filter estimation on two columns of same table
## What changes were proposed in this pull request?

In SQL queries, we also see predicate expressions involving two columns such as "column-1 (op) column-2" where column-1 and column-2 belong to same table. Note that, if column-1 and column-2 belong to different tables, then it is a join operator's work, NOT a filter operator's work.

This PR estimates filter selectivity on two columns of same table.  For example, multiple tpc-h queries have this predicate "WHERE l_commitdate < l_receiptdate"

## How was this patch tested?

We added 6 new test cases to test various logical predicates involving two columns of same table.

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

Author: Ron Hu <ron.hu@huawei.com>
Author: U-CHINA\r00754707 <r00754707@R00754707-SC04.china.huawei.com>

Closes #17415 from ron8hu/filterTwoColumns.
2017-04-03 17:27:12 -07:00
Adrian Ionescu 703c42c398 [SPARK-20194] Add support for partition pruning to in-memory catalog
## What changes were proposed in this pull request?
This patch implements `listPartitionsByFilter()` for `InMemoryCatalog` and thus resolves an outstanding TODO causing the `PruneFileSourcePartitions` optimizer rule not to apply when "spark.sql.catalogImplementation" is set to "in-memory" (which is the default).

The change is straightforward: it extracts the code for further filtering of the list of partitions returned by the metastore's `getPartitionsByFilter()` out from `HiveExternalCatalog` into `ExternalCatalogUtils` and calls this new function from `InMemoryCatalog` on the whole list of partitions.

Now that this method is implemented we can always pass the `CatalogTable` to the `DataSource` in `FindDataSourceTable`, so that the latter is resolved to a relation with a `CatalogFileIndex`, which is what the `PruneFileSourcePartitions` rule matches for.

## How was this patch tested?
Ran existing tests and added new test for `listPartitionsByFilter` in `ExternalCatalogSuite`, which is subclassed by both `InMemoryCatalogSuite` and `HiveExternalCatalogSuite`.

Author: Adrian Ionescu <adrian@databricks.com>

Closes #17510 from adrian-ionescu/InMemoryCatalog.
2017-04-03 08:48:49 -07:00
hyukjinkwon d40cbb8618 [SPARK-20143][SQL] DataType.fromJson should throw an exception with better message
## What changes were proposed in this pull request?

Currently, `DataType.fromJson` throws `scala.MatchError` or `java.util.NoSuchElementException` in some cases when the JSON input is invalid as below:

```scala
DataType.fromJson(""""abcd"""")
```

```
java.util.NoSuchElementException: key not found: abcd
  at ...
```

```scala
DataType.fromJson("""{"abcd":"a"}""")
```

```
scala.MatchError: JObject(List((abcd,JString(a)))) (of class org.json4s.JsonAST$JObject)
  at ...
```

```scala
DataType.fromJson("""{"fields": [{"a":123}], "type": "struct"}""")
```

```
scala.MatchError: JObject(List((a,JInt(123)))) (of class org.json4s.JsonAST$JObject)
  at ...
```

After this PR,

```scala
DataType.fromJson(""""abcd"""")
```

```
java.lang.IllegalArgumentException: Failed to convert the JSON string 'abcd' to a data type.
  at ...
```

```scala
DataType.fromJson("""{"abcd":"a"}""")
```

```
java.lang.IllegalArgumentException: Failed to convert the JSON string '{"abcd":"a"}' to a data type.
  at ...
```

```scala
DataType.fromJson("""{"fields": [{"a":123}], "type": "struct"}""")
  at ...
```

```
java.lang.IllegalArgumentException: Failed to convert the JSON string '{"a":123}' to a field.
```

## How was this patch tested?

Unit test added in `DataTypeSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17468 from HyukjinKwon/fromjson_exception.
2017-04-02 07:26:49 -07:00
wangzhenhua 2287f3d0b8 [SPARK-20186][SQL] BroadcastHint should use child's stats
## What changes were proposed in this pull request?

`BroadcastHint` should use child's statistics and set `isBroadcastable` to true.

## How was this patch tested?

Added a new stats estimation test for `BroadcastHint`.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17504 from wzhfy/broadcastHintEstimation.
2017-04-01 22:19:08 +08:00
Jacek Laskowski 0197262a35 [DOCS] Docs-only improvements
…adoc

## What changes were proposed in this pull request?

Use recommended values for row boundaries in Window's scaladoc, i.e. `Window.unboundedPreceding`, `Window.unboundedFollowing`, and `Window.currentRow` (that were introduced in 2.1.0).

## How was this patch tested?

Local build

Author: Jacek Laskowski <jacek@japila.pl>

Closes #17417 from jaceklaskowski/window-expression-scaladoc.
2017-03-30 16:07:27 +01:00
Xiao Li 5c8ef376e8 [SPARK-17075][SQL][FOLLOWUP] Add Estimation of Constant Literal
### What changes were proposed in this pull request?
`FalseLiteral` and `TrueLiteral` should have been eliminated by optimizer rule `BooleanSimplification`, but null literals might be added by optimizer rule `NullPropagation`. For safety, our filter estimation should handle all the eligible literal cases.

Our optimizer rule BooleanSimplification is unable to remove the null literal in many cases. For example, `a < 0 or null`. Thus, we need to handle null literal in filter estimation.

`Not` can be pushed down below `And` and `Or`. Then, we could see two consecutive `Not`, which need to be collapsed into one. Because of the limited expression support for filter estimation, we just need to handle the case `Not(null)` for avoiding incorrect error due to the boolean operation on null. For details, see below matrix.

```
not NULL = NULL
NULL or false = NULL
NULL or true = true
NULL or NULL = NULL
NULL and false = false
NULL and true = NULL
NULL and NULL = NULL
```
### How was this patch tested?
Added the test cases.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17446 from gatorsmile/constantFilterEstimation.
2017-03-29 12:43:22 -07:00
Takeshi Yamamuro c4008480b7 [SPARK-20009][SQL] Support DDL strings for defining schema in functions.from_json
## What changes were proposed in this pull request?
This pr added `StructType.fromDDL`  to convert a DDL format string into `StructType` for defining schemas in `functions.from_json`.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17406 from maropu/SPARK-20009.
2017-03-29 12:37:49 -07:00
wangzhenhua 4fcc214d9e [SPARK-20124][SQL] Join reorder should keep the same order of final project attributes
## What changes were proposed in this pull request?

Join reorder algorithm should keep exactly the same order of output attributes in the top project.
For example, if user want to select a, b, c, after reordering, we should output a, b, c in the same order as specified by user, instead of b, a, c or other orders.

## How was this patch tested?

A new test case is added in `JoinReorderSuite`.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17453 from wzhfy/keepOrderInProject.
2017-03-28 22:22:38 +08:00
wangzhenhua 91559d277f [SPARK-20094][SQL] Preventing push down of IN subquery to Join operator
## What changes were proposed in this pull request?

TPCDS q45 fails becuase:
`ReorderJoin` collects all predicates and try to put them into join condition when creating ordered join. If a predicate with an IN subquery (`ListQuery`) is in a join condition instead of a filter condition, `RewritePredicateSubquery.rewriteExistentialExpr` would fail to convert the subquery to an `ExistenceJoin`, and thus result in error.

We should prevent push down of IN subquery to Join operator.

## How was this patch tested?

Add a new test case in `FilterPushdownSuite`.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17428 from wzhfy/noSubqueryInJoinCond.
2017-03-28 13:43:23 +02:00
Michal Senkyr 6c70a38c2e [SPARK-19088][SQL] Optimize sequence type deserialization codegen
## What changes were proposed in this pull request?

Optimization of arbitrary Scala sequence deserialization introduced by #16240.

The previous implementation constructed an array which was then converted by `to`. This required two passes in most cases.

This implementation attempts to remedy that by using `Builder`s provided by the `newBuilder` method on every Scala collection's companion object to build the resulting collection directly.

Example codegen for simple `List` (obtained using `Seq(List(1)).toDS().map(identity).queryExecution.debug.codegen`):

Before:

```
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIterator(references);
/* 003 */ }
/* 004 */
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private scala.collection.Iterator inputadapter_input;
/* 009 */   private boolean deserializetoobject_resultIsNull;
/* 010 */   private java.lang.Object[] deserializetoobject_argValue;
/* 011 */   private boolean MapObjects_loopIsNull1;
/* 012 */   private int MapObjects_loopValue0;
/* 013 */   private boolean deserializetoobject_resultIsNull1;
/* 014 */   private scala.collection.generic.CanBuildFrom deserializetoobject_argValue1;
/* 015 */   private UnsafeRow deserializetoobject_result;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder deserializetoobject_holder;
/* 017 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter deserializetoobject_rowWriter;
/* 018 */   private scala.collection.immutable.List mapelements_argValue;
/* 019 */   private UnsafeRow mapelements_result;
/* 020 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder mapelements_holder;
/* 021 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter mapelements_rowWriter;
/* 022 */   private scala.collection.immutable.List serializefromobject_argValue;
/* 023 */   private UnsafeRow serializefromobject_result;
/* 024 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 025 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 026 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter serializefromobject_arrayWriter;
/* 027 */
/* 028 */   public GeneratedIterator(Object[] references) {
/* 029 */     this.references = references;
/* 030 */   }
/* 031 */
/* 032 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 033 */     partitionIndex = index;
/* 034 */     this.inputs = inputs;
/* 035 */     inputadapter_input = inputs[0];
/* 036 */
/* 037 */     deserializetoobject_result = new UnsafeRow(1);
/* 038 */     this.deserializetoobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(deserializetoobject_result, 32);
/* 039 */     this.deserializetoobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(deserializetoobject_holder, 1);
/* 040 */
/* 041 */     mapelements_result = new UnsafeRow(1);
/* 042 */     this.mapelements_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(mapelements_result, 32);
/* 043 */     this.mapelements_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(mapelements_holder, 1);
/* 044 */
/* 045 */     serializefromobject_result = new UnsafeRow(1);
/* 046 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 32);
/* 047 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 048 */     this.serializefromobject_arrayWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
/* 049 */
/* 050 */   }
/* 051 */
/* 052 */   protected void processNext() throws java.io.IOException {
/* 053 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 054 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 055 */       ArrayData inputadapter_value = inputadapter_row.getArray(0);
/* 056 */
/* 057 */       deserializetoobject_resultIsNull = false;
/* 058 */
/* 059 */       if (!deserializetoobject_resultIsNull) {
/* 060 */         ArrayData deserializetoobject_value3 = null;
/* 061 */
/* 062 */         if (!false) {
/* 063 */           Integer[] deserializetoobject_convertedArray = null;
/* 064 */           int deserializetoobject_dataLength = inputadapter_value.numElements();
/* 065 */           deserializetoobject_convertedArray = new Integer[deserializetoobject_dataLength];
/* 066 */
/* 067 */           int deserializetoobject_loopIndex = 0;
/* 068 */           while (deserializetoobject_loopIndex < deserializetoobject_dataLength) {
/* 069 */             MapObjects_loopValue0 = (int) (inputadapter_value.getInt(deserializetoobject_loopIndex));
/* 070 */             MapObjects_loopIsNull1 = inputadapter_value.isNullAt(deserializetoobject_loopIndex);
/* 071 */
/* 072 */             if (MapObjects_loopIsNull1) {
/* 073 */               throw new RuntimeException(((java.lang.String) references[0]));
/* 074 */             }
/* 075 */             if (false) {
/* 076 */               deserializetoobject_convertedArray[deserializetoobject_loopIndex] = null;
/* 077 */             } else {
/* 078 */               deserializetoobject_convertedArray[deserializetoobject_loopIndex] = MapObjects_loopValue0;
/* 079 */             }
/* 080 */
/* 081 */             deserializetoobject_loopIndex += 1;
/* 082 */           }
/* 083 */
/* 084 */           deserializetoobject_value3 = new org.apache.spark.sql.catalyst.util.GenericArrayData(deserializetoobject_convertedArray);
/* 085 */         }
/* 086 */         boolean deserializetoobject_isNull2 = true;
/* 087 */         java.lang.Object[] deserializetoobject_value2 = null;
/* 088 */         if (!false) {
/* 089 */           deserializetoobject_isNull2 = false;
/* 090 */           if (!deserializetoobject_isNull2) {
/* 091 */             Object deserializetoobject_funcResult = null;
/* 092 */             deserializetoobject_funcResult = deserializetoobject_value3.array();
/* 093 */             if (deserializetoobject_funcResult == null) {
/* 094 */               deserializetoobject_isNull2 = true;
/* 095 */             } else {
/* 096 */               deserializetoobject_value2 = (java.lang.Object[]) deserializetoobject_funcResult;
/* 097 */             }
/* 098 */
/* 099 */           }
/* 100 */           deserializetoobject_isNull2 = deserializetoobject_value2 == null;
/* 101 */         }
/* 102 */         deserializetoobject_resultIsNull = deserializetoobject_isNull2;
/* 103 */         deserializetoobject_argValue = deserializetoobject_value2;
/* 104 */       }
/* 105 */
/* 106 */       boolean deserializetoobject_isNull1 = deserializetoobject_resultIsNull;
/* 107 */       final scala.collection.Seq deserializetoobject_value1 = deserializetoobject_resultIsNull ? null : scala.collection.mutable.WrappedArray.make(deserializetoobject_argValue);
/* 108 */       deserializetoobject_isNull1 = deserializetoobject_value1 == null;
/* 109 */       boolean deserializetoobject_isNull = true;
/* 110 */       scala.collection.immutable.List deserializetoobject_value = null;
/* 111 */       if (!deserializetoobject_isNull1) {
/* 112 */         deserializetoobject_resultIsNull1 = false;
/* 113 */
/* 114 */         if (!deserializetoobject_resultIsNull1) {
/* 115 */           boolean deserializetoobject_isNull6 = false;
/* 116 */           final scala.collection.generic.CanBuildFrom deserializetoobject_value6 = false ? null : scala.collection.immutable.List.canBuildFrom();
/* 117 */           deserializetoobject_isNull6 = deserializetoobject_value6 == null;
/* 118 */           deserializetoobject_resultIsNull1 = deserializetoobject_isNull6;
/* 119 */           deserializetoobject_argValue1 = deserializetoobject_value6;
/* 120 */         }
/* 121 */
/* 122 */         deserializetoobject_isNull = deserializetoobject_resultIsNull1;
/* 123 */         if (!deserializetoobject_isNull) {
/* 124 */           Object deserializetoobject_funcResult1 = null;
/* 125 */           deserializetoobject_funcResult1 = deserializetoobject_value1.to(deserializetoobject_argValue1);
/* 126 */           if (deserializetoobject_funcResult1 == null) {
/* 127 */             deserializetoobject_isNull = true;
/* 128 */           } else {
/* 129 */             deserializetoobject_value = (scala.collection.immutable.List) deserializetoobject_funcResult1;
/* 130 */           }
/* 131 */
/* 132 */         }
/* 133 */         deserializetoobject_isNull = deserializetoobject_value == null;
/* 134 */       }
/* 135 */
/* 136 */       boolean mapelements_isNull = true;
/* 137 */       scala.collection.immutable.List mapelements_value = null;
/* 138 */       if (!false) {
/* 139 */         mapelements_argValue = deserializetoobject_value;
/* 140 */
/* 141 */         mapelements_isNull = false;
/* 142 */         if (!mapelements_isNull) {
/* 143 */           Object mapelements_funcResult = null;
/* 144 */           mapelements_funcResult = ((scala.Function1) references[1]).apply(mapelements_argValue);
/* 145 */           if (mapelements_funcResult == null) {
/* 146 */             mapelements_isNull = true;
/* 147 */           } else {
/* 148 */             mapelements_value = (scala.collection.immutable.List) mapelements_funcResult;
/* 149 */           }
/* 150 */
/* 151 */         }
/* 152 */         mapelements_isNull = mapelements_value == null;
/* 153 */       }
/* 154 */
/* 155 */       if (mapelements_isNull) {
/* 156 */         throw new RuntimeException(((java.lang.String) references[2]));
/* 157 */       }
/* 158 */       serializefromobject_argValue = mapelements_value;
/* 159 */
/* 160 */       final ArrayData serializefromobject_value = false ? null : new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_argValue);
/* 161 */       serializefromobject_holder.reset();
/* 162 */
/* 163 */       // Remember the current cursor so that we can calculate how many bytes are
/* 164 */       // written later.
/* 165 */       final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
/* 166 */
/* 167 */       if (serializefromobject_value instanceof UnsafeArrayData) {
/* 168 */         final int serializefromobject_sizeInBytes = ((UnsafeArrayData) serializefromobject_value).getSizeInBytes();
/* 169 */         // grow the global buffer before writing data.
/* 170 */         serializefromobject_holder.grow(serializefromobject_sizeInBytes);
/* 171 */         ((UnsafeArrayData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 172 */         serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
/* 173 */
/* 174 */       } else {
/* 175 */         final int serializefromobject_numElements = serializefromobject_value.numElements();
/* 176 */         serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 4);
/* 177 */
/* 178 */         for (int serializefromobject_index = 0; serializefromobject_index < serializefromobject_numElements; serializefromobject_index++) {
/* 179 */           if (serializefromobject_value.isNullAt(serializefromobject_index)) {
/* 180 */             serializefromobject_arrayWriter.setNullInt(serializefromobject_index);
/* 181 */           } else {
/* 182 */             final int serializefromobject_element = serializefromobject_value.getInt(serializefromobject_index);
/* 183 */             serializefromobject_arrayWriter.write(serializefromobject_index, serializefromobject_element);
/* 184 */           }
/* 185 */         }
/* 186 */       }
/* 187 */
/* 188 */       serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
/* 189 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
/* 190 */       append(serializefromobject_result);
/* 191 */       if (shouldStop()) return;
/* 192 */     }
/* 193 */   }
/* 194 */ }
```

After:

```
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIterator(references);
/* 003 */ }
/* 004 */
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private scala.collection.Iterator inputadapter_input;
/* 009 */   private boolean CollectObjects_loopIsNull1;
/* 010 */   private int CollectObjects_loopValue0;
/* 011 */   private UnsafeRow deserializetoobject_result;
/* 012 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder deserializetoobject_holder;
/* 013 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter deserializetoobject_rowWriter;
/* 014 */   private scala.collection.immutable.List mapelements_argValue;
/* 015 */   private UnsafeRow mapelements_result;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder mapelements_holder;
/* 017 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter mapelements_rowWriter;
/* 018 */   private scala.collection.immutable.List serializefromobject_argValue;
/* 019 */   private UnsafeRow serializefromobject_result;
/* 020 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 021 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 022 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter serializefromobject_arrayWriter;
/* 023 */
/* 024 */   public GeneratedIterator(Object[] references) {
/* 025 */     this.references = references;
/* 026 */   }
/* 027 */
/* 028 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 029 */     partitionIndex = index;
/* 030 */     this.inputs = inputs;
/* 031 */     inputadapter_input = inputs[0];
/* 032 */
/* 033 */     deserializetoobject_result = new UnsafeRow(1);
/* 034 */     this.deserializetoobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(deserializetoobject_result, 32);
/* 035 */     this.deserializetoobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(deserializetoobject_holder, 1);
/* 036 */
/* 037 */     mapelements_result = new UnsafeRow(1);
/* 038 */     this.mapelements_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(mapelements_result, 32);
/* 039 */     this.mapelements_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(mapelements_holder, 1);
/* 040 */
/* 041 */     serializefromobject_result = new UnsafeRow(1);
/* 042 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 32);
/* 043 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 044 */     this.serializefromobject_arrayWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
/* 045 */
/* 046 */   }
/* 047 */
/* 048 */   protected void processNext() throws java.io.IOException {
/* 049 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 050 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 051 */       ArrayData inputadapter_value = inputadapter_row.getArray(0);
/* 052 */
/* 053 */       scala.collection.immutable.List deserializetoobject_value = null;
/* 054 */
/* 055 */       if (!false) {
/* 056 */         int deserializetoobject_dataLength = inputadapter_value.numElements();
/* 057 */         scala.collection.mutable.Builder CollectObjects_builderValue2 = scala.collection.immutable.List$.MODULE$.newBuilder();
/* 058 */         CollectObjects_builderValue2.sizeHint(deserializetoobject_dataLength);
/* 059 */
/* 060 */         int deserializetoobject_loopIndex = 0;
/* 061 */         while (deserializetoobject_loopIndex < deserializetoobject_dataLength) {
/* 062 */           CollectObjects_loopValue0 = (int) (inputadapter_value.getInt(deserializetoobject_loopIndex));
/* 063 */           CollectObjects_loopIsNull1 = inputadapter_value.isNullAt(deserializetoobject_loopIndex);
/* 064 */
/* 065 */           if (CollectObjects_loopIsNull1) {
/* 066 */             throw new RuntimeException(((java.lang.String) references[0]));
/* 067 */           }
/* 068 */           if (false) {
/* 069 */             CollectObjects_builderValue2.$plus$eq(null);
/* 070 */           } else {
/* 071 */             CollectObjects_builderValue2.$plus$eq(CollectObjects_loopValue0);
/* 072 */           }
/* 073 */
/* 074 */           deserializetoobject_loopIndex += 1;
/* 075 */         }
/* 076 */
/* 077 */         deserializetoobject_value = (scala.collection.immutable.List) CollectObjects_builderValue2.result();
/* 078 */       }
/* 079 */
/* 080 */       boolean mapelements_isNull = true;
/* 081 */       scala.collection.immutable.List mapelements_value = null;
/* 082 */       if (!false) {
/* 083 */         mapelements_argValue = deserializetoobject_value;
/* 084 */
/* 085 */         mapelements_isNull = false;
/* 086 */         if (!mapelements_isNull) {
/* 087 */           Object mapelements_funcResult = null;
/* 088 */           mapelements_funcResult = ((scala.Function1) references[1]).apply(mapelements_argValue);
/* 089 */           if (mapelements_funcResult == null) {
/* 090 */             mapelements_isNull = true;
/* 091 */           } else {
/* 092 */             mapelements_value = (scala.collection.immutable.List) mapelements_funcResult;
/* 093 */           }
/* 094 */
/* 095 */         }
/* 096 */         mapelements_isNull = mapelements_value == null;
/* 097 */       }
/* 098 */
/* 099 */       if (mapelements_isNull) {
/* 100 */         throw new RuntimeException(((java.lang.String) references[2]));
/* 101 */       }
/* 102 */       serializefromobject_argValue = mapelements_value;
/* 103 */
/* 104 */       final ArrayData serializefromobject_value = false ? null : new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_argValue);
/* 105 */       serializefromobject_holder.reset();
/* 106 */
/* 107 */       // Remember the current cursor so that we can calculate how many bytes are
/* 108 */       // written later.
/* 109 */       final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
/* 110 */
/* 111 */       if (serializefromobject_value instanceof UnsafeArrayData) {
/* 112 */         final int serializefromobject_sizeInBytes = ((UnsafeArrayData) serializefromobject_value).getSizeInBytes();
/* 113 */         // grow the global buffer before writing data.
/* 114 */         serializefromobject_holder.grow(serializefromobject_sizeInBytes);
/* 115 */         ((UnsafeArrayData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 116 */         serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
/* 117 */
/* 118 */       } else {
/* 119 */         final int serializefromobject_numElements = serializefromobject_value.numElements();
/* 120 */         serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 4);
/* 121 */
/* 122 */         for (int serializefromobject_index = 0; serializefromobject_index < serializefromobject_numElements; serializefromobject_index++) {
/* 123 */           if (serializefromobject_value.isNullAt(serializefromobject_index)) {
/* 124 */             serializefromobject_arrayWriter.setNullInt(serializefromobject_index);
/* 125 */           } else {
/* 126 */             final int serializefromobject_element = serializefromobject_value.getInt(serializefromobject_index);
/* 127 */             serializefromobject_arrayWriter.write(serializefromobject_index, serializefromobject_element);
/* 128 */           }
/* 129 */         }
/* 130 */       }
/* 131 */
/* 132 */       serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
/* 133 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
/* 134 */       append(serializefromobject_result);
/* 135 */       if (shouldStop()) return;
/* 136 */     }
/* 137 */   }
/* 138 */ }
```

Benchmark results before:

```
OpenJDK 64-Bit Server VM 1.8.0_112-b15 on Linux 4.8.13-1-ARCH
AMD A10-4600M APU with Radeon(tm) HD Graphics
collect:                                 Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Seq                                            269 /  370          0.0      269125.8       1.0X
List                                           154 /  176          0.0      154453.5       1.7X
mutable.Queue                                  210 /  233          0.0      209691.6       1.3X
```

Benchmark results after:

```
OpenJDK 64-Bit Server VM 1.8.0_112-b15 on Linux 4.8.13-1-ARCH
AMD A10-4600M APU with Radeon(tm) HD Graphics
collect:                                 Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Seq                                            255 /  316          0.0      254697.3       1.0X
List                                           152 /  177          0.0      152410.0       1.7X
mutable.Queue                                  213 /  235          0.0      213470.0       1.2X
```

## How was this patch tested?

```bash
./build/mvn -DskipTests clean package && ./dev/run-tests
```

Additionally in Spark Shell:

```scala
case class QueueClass(q: scala.collection.immutable.Queue[Int])

spark.createDataset(Seq(List(1,2,3))).map(x => QueueClass(scala.collection.immutable.Queue(x: _*))).map(_.q.dequeue).collect
```

Author: Michal Senkyr <mike.senkyr@gmail.com>

Closes #16541 from michalsenkyr/dataset-seq-builder.
2017-03-28 10:09:49 +08:00
Herman van Hovell ea361165e1 [SPARK-20100][SQL] Refactor SessionState initialization
## What changes were proposed in this pull request?
The current SessionState initialization code path is quite complex. A part of the creation is done in the SessionState companion objects, a part of the creation is one inside the SessionState class, and a part is done by passing functions.

This PR refactors this code path, and consolidates SessionState initialization into a builder class. This SessionState will not do any initialization and just becomes a place holder for the various Spark SQL internals. This also lays the ground work for two future improvements:

1. This provides us with a start for removing the `HiveSessionState`. Removing the `HiveSessionState` would also require us to move resource loading into a separate class, and to (re)move metadata hive.
2. This makes it easier to customize the Spark Session. Currently you will need to create a custom version of the builder. I have added hooks to facilitate this. A future step will be to create a semi stable API on top of this.

## How was this patch tested?
Existing tests.

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

Closes #17433 from hvanhovell/SPARK-20100.
2017-03-28 10:07:24 +08:00
wangzhenhua 890493458d [SPARK-20104][SQL] Don't estimate IsNull or IsNotNull predicates for non-leaf node
## What changes were proposed in this pull request?

In current stage, we don't have advanced statistics such as sketches or histograms. As a result, some operator can't estimate `nullCount` accurately. E.g. left outer join estimation does not accurately update `nullCount` currently. So for `IsNull` and `IsNotNull` predicates, we only estimate them when the child is a leaf node, whose `nullCount` is accurate.

## How was this patch tested?

A new test case is added in `FilterEstimationSuite`.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17438 from wzhfy/nullEstimation.
2017-03-27 23:41:27 +08:00
Herman van Hovell 617ab6445e [SPARK-20086][SQL] CollapseWindow should not collapse dependent adjacent windows
## What changes were proposed in this pull request?
The `CollapseWindow` is currently to aggressive when collapsing adjacent windows. It also collapses windows in the which the parent produces a column that is consumed by the child; this creates an invalid window which will fail at runtime.

This PR fixes this by adding a check for dependent adjacent windows to the `CollapseWindow` rule.

## How was this patch tested?
Added a new test case to `CollapseWindowSuite`

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

Closes #17432 from hvanhovell/SPARK-20086.
2017-03-26 22:47:31 +02:00
Liang-Chi Hsieh e011004bed [SPARK-19846][SQL] Add a flag to disable constraint propagation
## What changes were proposed in this pull request?

Constraint propagation can be computation expensive and block the driver execution for long time. For example, the below benchmark needs 30mins.

Compared with previous PRs #16998, #16785, this is a much simpler option: add a flag to disable constraint propagation.

### Benchmark

Run the following codes locally.

    import org.apache.spark.ml.{Pipeline, PipelineStage}
    import org.apache.spark.ml.feature.{OneHotEncoder, StringIndexer, VectorAssembler}
    import org.apache.spark.sql.internal.SQLConf

    spark.conf.set(SQLConf.CONSTRAINT_PROPAGATION_ENABLED.key, false)

    val df = (1 to 40).foldLeft(Seq((1, "foo"), (2, "bar"), (3, "baz")).toDF("id", "x0"))((df, i) => df.withColumn(s"x$i", $"x0"))

    val indexers = df.columns.tail.map(c => new StringIndexer()
      .setInputCol(c)
      .setOutputCol(s"${c}_indexed")
      .setHandleInvalid("skip"))

    val encoders = indexers.map(indexer => new OneHotEncoder()
      .setInputCol(indexer.getOutputCol)
      .setOutputCol(s"${indexer.getOutputCol}_encoded")
      .setDropLast(true))

    val stages: Array[PipelineStage] = indexers ++ encoders
    val pipeline = new Pipeline().setStages(stages)

    val startTime = System.nanoTime
    pipeline.fit(df).transform(df).show
    val runningTime = System.nanoTime - startTime

Before this patch: 1786001 ms ~= 30 mins
After this patch: 26392 ms = less than half of a minute

Related PRs: #16998, #16785.

## How was this patch tested?

Jenkins tests.

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

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

Closes #17186 from viirya/add-flag-disable-constraint-propagation.
2017-03-25 00:04:51 +01:00
Tathagata Das 82b598b963 [SPARK-20057][SS] Renamed KeyedState to GroupState in mapGroupsWithState
## What changes were proposed in this pull request?

Since the state is tied a "group" in the "mapGroupsWithState" operations, its better to call the state "GroupState" instead of a key. This would make it more general if you extends this operation to RelationGroupedDataset and python APIs.

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

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

Closes #17385 from tdas/SPARK-20057.
2017-03-22 12:30:36 -07:00
hyukjinkwon 465818389a [SPARK-19949][SQL][FOLLOW-UP] Clean up parse modes and update related comments
## What changes were proposed in this pull request?

This PR proposes to make `mode` options in both CSV and JSON to use `cass object` and fix some related comments related previous fix.

Also, this PR modifies some tests related parse modes.

## How was this patch tested?

Modified unit tests in both `CSVSuite.scala` and `JsonSuite.scala`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17377 from HyukjinKwon/SPARK-19949.
2017-03-22 09:52:37 -07:00
Tathagata Das c1e87e384d [SPARK-20030][SS] Event-time-based timeout for MapGroupsWithState
## What changes were proposed in this pull request?

Adding event time based timeout. The user sets the timeout timestamp directly using `KeyedState.setTimeoutTimestamp`. The keys times out when the watermark crosses the timeout timestamp.

## How was this patch tested?
Unit tests

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

Closes #17361 from tdas/SPARK-20030.
2017-03-21 21:27:08 -07:00
zhaorongsheng 7dbc162f12 [SPARK-20017][SQL] change the nullability of function 'StringToMap' from 'false' to 'true'
## What changes were proposed in this pull request?

Change the nullability of function `StringToMap` from `false` to `true`.

Author: zhaorongsheng <334362872@qq.com>

Closes #17350 from zhaorongsheng/bug-fix_strToMap_NPE.
2017-03-21 11:30:55 -07:00
Xin Wu 4c0ff5f585 [SPARK-19261][SQL] Alter add columns for Hive serde and some datasource tables
## What changes were proposed in this pull request?
Support` ALTER TABLE ADD COLUMNS (...) `syntax for Hive serde and some datasource tables.
In this PR, we consider a few aspects:

1. View is not supported for `ALTER ADD COLUMNS`

2. Since tables created in SparkSQL with Hive DDL syntax will populate table properties with schema information, we need make sure the consistency of the schema before and after ALTER operation in order for future use.

3. For embedded-schema type of format, such as `parquet`, we need to make sure that the predicate on the newly-added columns can be evaluated properly, or pushed down properly. In case of the data file does not have the columns for the newly-added columns, such predicates should return as if the column values are NULLs.

4. For datasource table, this feature does not support the following:
4.1 TEXT format, since there is only one default column `value` is inferred for text format data.
4.2 ORC format, since SparkSQL native ORC reader does not support the difference between user-specified-schema and inferred schema from ORC files.
4.3 Third party datasource types that implements RelationProvider, including the built-in JDBC format, since different implementations by the vendors may have different ways to dealing with schema.
4.4 Other datasource types, such as `parquet`, `json`, `csv`, `hive` are supported.

5. Column names being added can not be duplicate of any existing data column or partition column names. Case sensitivity is taken into consideration according to the sql configuration.

6. This feature also supports In-Memory catalog, while Hive support is turned off.
## How was this patch tested?
Add new test cases

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

Closes #16626 from xwu0226/alter_add_columns.
2017-03-21 08:49:54 -07:00
Xiao Li d2dcd6792f [SPARK-20024][SQL][TEST-MAVEN] SessionCatalog reset need to set the current database of ExternalCatalog
### What changes were proposed in this pull request?
SessionCatalog API setCurrentDatabase does not set the current database of the underlying ExternalCatalog. Thus, weird errors could come in the test suites after we call reset. We need to fix it.

So far, have not found the direct impact in the other code paths because we expect all the SessionCatalog APIs should always use the current database value we managed, unless some of code paths skip it. Thus, we fix it in the test-only function reset().

### How was this patch tested?
Multiple test case failures are observed in mvn and add a test case in SessionCatalogSuite.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17354 from gatorsmile/useDB.
2017-03-20 22:52:45 -07:00
Zheng RuiFeng 10691d36de [SPARK-19573][SQL] Make NaN/null handling consistent in approxQuantile
## What changes were proposed in this pull request?
update `StatFunctions.multipleApproxQuantiles` to handle NaN/null

## How was this patch tested?
existing tests and added tests

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #16971 from zhengruifeng/quantiles_nan.
2017-03-20 18:25:59 -07:00
Ioana Delaney 8163911594 [SPARK-17791][SQL] Join reordering using star schema detection
## What changes were proposed in this pull request?

Star schema consists of one or more fact tables referencing a number of dimension tables. In general, queries against star schema are expected to run fast because of the established RI constraints among the tables. This design proposes a join reordering based on natural, generally accepted heuristics for star schema queries:
- Finds the star join with the largest fact table and places it on the driving arm of the left-deep join. This plan avoids large tables on the inner, and thus favors hash joins.
- Applies the most selective dimensions early in the plan to reduce the amount of data flow.

The design document was included in SPARK-17791.

Link to the google doc: [StarSchemaDetection](https://docs.google.com/document/d/1UAfwbm_A6wo7goHlVZfYK99pqDMEZUumi7pubJXETEA/edit?usp=sharing)

## How was this patch tested?

A new test suite StarJoinSuite.scala was implemented.

Author: Ioana Delaney <ioanamdelaney@gmail.com>

Closes #15363 from ioana-delaney/starJoinReord2.
2017-03-20 16:04:58 +08:00
hyukjinkwon 0cdcf91145 [SPARK-19849][SQL] Support ArrayType in to_json to produce JSON array
## What changes were proposed in this pull request?

This PR proposes to support an array of struct type in `to_json` as below:

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

val df = Seq(Tuple1(Tuple1(1) :: Nil)).toDF("a")
df.select(to_json($"a").as("json")).show()
```

```
+----------+
|      json|
+----------+
|[{"_1":1}]|
+----------+
```

Currently, it throws an exception as below (a newline manually inserted for readability):

```
org.apache.spark.sql.AnalysisException: cannot resolve 'structtojson(`array`)' due to data type
mismatch: structtojson requires that the expression is a struct expression.;;
```

This allows the roundtrip with `from_json` as below:

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

val schema = ArrayType(StructType(StructField("a", IntegerType) :: Nil))
val df = Seq("""[{"a":1}, {"a":2}]""").toDF("json").select(from_json($"json", schema).as("array"))
df.show()

// Read back.
df.select(to_json($"array").as("json")).show()
```

```
+----------+
|     array|
+----------+
|[[1], [2]]|
+----------+

+-----------------+
|             json|
+-----------------+
|[{"a":1},{"a":2}]|
+-----------------+
```

Also, this PR proposes to rename from `StructToJson` to `StructsToJson ` and `JsonToStruct` to `JsonToStructs`.

## How was this patch tested?

Unit tests in `JsonFunctionsSuite` and `JsonExpressionsSuite` for Scala, doctest for Python and test in `test_sparkSQL.R` for R.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17192 from HyukjinKwon/SPARK-19849.
2017-03-19 22:33:01 -07:00
Tathagata Das 990af630d0 [SPARK-19067][SS] Processing-time-based timeout in MapGroupsWithState
## What changes were proposed in this pull request?

When a key does not get any new data in `mapGroupsWithState`, the mapping function is never called on it. So we need a timeout feature that calls the function again in such cases, so that the user can decide whether to continue waiting or clean up (remove state, save stuff externally, etc.).
Timeouts can be either based on processing time or event time. This JIRA is for processing time, but defines the high level API design for both. The usage would look like this.
```
def stateFunction(key: K, value: Iterator[V], state: KeyedState[S]): U = {
  ...
  state.setTimeoutDuration(10000)
  ...
}

dataset					// type is Dataset[T]
  .groupByKey[K](keyingFunc)   // generates KeyValueGroupedDataset[K, T]
  .mapGroupsWithState[S, U](
     func = stateFunction,
     timeout = KeyedStateTimeout.withProcessingTime)	// returns Dataset[U]
```

Note the following design aspects.

- The timeout type is provided as a param in mapGroupsWithState as a parameter global to all the keys. This is so that the planner knows this at planning time, and accordingly optimize the execution based on whether to saves extra info in state or not (e.g. timeout durations or timestamps).

- The exact timeout duration is provided inside the function call so that it can be customized on a per key basis.

- When the timeout occurs for a key, the function is called with no values, and KeyedState.isTimingOut() set to true.

- The timeout is reset for key every time the function is called on the key, that is, when the key has new data, or the key has timed out. So the user has to set the timeout duration everytime the function is called, otherwise there will not be any timeout set.

Guarantees provided on timeout of key, when timeout duration is D ms:
- Timeout will never be called before real clock time has advanced by D ms
- Timeout will be called eventually when there is a trigger with any data in it (i.e. after D ms). So there is a no strict upper bound on when the timeout would occur. For example, if there is no data in the stream (for any key) for a while, then the timeout will not be hit.

Implementation details:
- Added new param to `mapGroupsWithState` for timeout
- Added new method to `StateStore` to filter data based on timeout timestamp
- Changed the internal map type of `HDFSBackedStateStore` from Java's `HashMap` to `ConcurrentHashMap` as the latter allows weakly-consistent fail-safe iterators on the map data. See comments in code for more details.
- Refactored logic of `MapGroupsWithStateExec` to
  - Save timeout info to state store for each key that has data.
  - Then, filter states that should be timed out based on the current batch processing timestamp.
- Moved KeyedState for `o.a.s.sql` to `o.a.s.sql.streaming`. I remember that this was a feedback in the MapGroupsWithState PR that I had forgotten to address.

## How was this patch tested?
New unit tests in
- MapGroupsWithStateSuite for timeouts.
- StateStoreSuite for new APIs in StateStore.

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

Closes #17179 from tdas/mapgroupwithstate-timeout.
2017-03-19 14:07:49 -07:00
wangzhenhua c083b6b7de [SPARK-19915][SQL] Exclude cartesian product candidates to reduce the search space
## What changes were proposed in this pull request?

We have some concerns about removing size in the cost model [in the previous pr](https://github.com/apache/spark/pull/17240). It's a tradeoff between code structure and algorithm completeness. I tend to keep the size and thus create this new pr without changing cost model.

What this pr does:
1. We only consider consecutive inner joinable items, thus excluding cartesian products in reordering procedure. This significantly reduces the search space and memory overhead of memo. Otherwise every combination of items will exist in the memo.
2. This pr also includes a bug fix: if a leaf item is a project(_, child), current solution will miss the project.

## How was this patch tested?

Added test cases.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17286 from wzhfy/joinReorder3.
2017-03-18 14:07:25 +08:00
windpiger 8e8f898335 [SPARK-19945][SQL] add test suite for SessionCatalog with HiveExternalCatalog
## What changes were proposed in this pull request?

Currently `SessionCatalogSuite` is only for `InMemoryCatalog`, there is no suite for `HiveExternalCatalog`.
And there are some ddl function is not proper to test in `ExternalCatalogSuite`, because some logic are not full implement in `ExternalCatalog`, these ddl functions are full implement in `SessionCatalog`(e.g. merge the same logic from `ExternalCatalog` up to `SessionCatalog` ).
It is better to test it in `SessionCatalogSuite` for this situation.

So we should add a test suite for `SessionCatalog` with `HiveExternalCatalog`

The main change is that in `SessionCatalogSuite` add two functions:
`withBasicCatalog` and `withEmptyCatalog`
And replace the code like  `val catalog = new SessionCatalog(newBasicCatalog)` with above two functions

## How was this patch tested?
add `HiveExternalSessionCatalogSuite`

Author: windpiger <songjun@outlook.com>

Closes #17287 from windpiger/sessioncatalogsuit.
2017-03-16 11:34:13 -07:00
Xiao Li 1472cac4bb [SPARK-19830][SQL] Add parseTableSchema API to ParserInterface
### What changes were proposed in this pull request?

Specifying the table schema in DDL formats is needed for different scenarios. For example,
- [specifying the schema in SQL function `from_json` using DDL formats](https://issues.apache.org/jira/browse/SPARK-19637), which is suggested by marmbrus ,
- [specifying the customized JDBC data types](https://github.com/apache/spark/pull/16209).

These two PRs need users to use the JSON format to specify the table schema. This is not user friendly.

This PR is to provide a `parseTableSchema` API in `ParserInterface`.

### How was this patch tested?
Added a test suite `TableSchemaParserSuite`

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17171 from gatorsmile/parseDDLStmt.
2017-03-16 12:06:20 +08:00
Takuya UESHIN 7ded39c223 [SPARK-19817][SQL] Make it clear that timeZone option is a general option in DataFrameReader/Writer.
## What changes were proposed in this pull request?

As timezone setting can also affect partition values, it works for all formats, we should make it clear.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #17281 from ueshin/issues/SPARK-19817.
2017-03-14 13:57:23 -07:00
Herman van Hovell e04c05cf41 [SPARK-19933][SQL] Do not change output of a subquery
## What changes were proposed in this pull request?
The `RemoveRedundantAlias` rule can change the output attributes (the expression id's to be precise) of a query by eliminating the redundant alias producing them. This is no problem for a regular query, but can cause problems for correlated subqueries: The attributes produced by the subquery are used in the parent plan; changing them will break the parent plan.

This PR fixes this by wrapping a subquery in a `Subquery` top level node when it gets optimized. The `RemoveRedundantAlias` rule now recognizes `Subquery` and makes sure that the output attributes of the `Subquery` node are retained.

## How was this patch tested?
Added a test case to `RemoveRedundantAliasAndProjectSuite` and added a regression test to `SubquerySuite`.

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

Closes #17278 from hvanhovell/SPARK-19933.
2017-03-14 18:52:16 +01:00
Herman van Hovell a0b92f73fe [SPARK-19850][SQL] Allow the use of aliases in SQL function calls
## What changes were proposed in this pull request?
We currently cannot use aliases in SQL function calls. This is inconvenient when you try to create a struct. This SQL query for example `select struct(1, 2) st`, will create a struct with column names `col1` and `col2`. This is even more problematic when we want to append a field to an existing struct. For example if we want to a field to struct `st` we would issue the following SQL query `select struct(st.*, 1) as st from src`, the result will be struct `st` with an a column with a non descriptive name `col3` (if `st` itself has 2 fields).

This PR proposes to change this by allowing the use of aliased expression in function parameters. For example `select struct(1 as a, 2 as b) st`, will create a struct with columns `a` & `b`.

## How was this patch tested?
Added a test to `ExpressionParserSuite` and added a test file for `SQLQueryTestSuite`.

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

Closes #17245 from hvanhovell/SPARK-19850.
2017-03-14 12:49:30 +01:00
Nattavut Sutyanyong 4ce970d714 [SPARK-18874][SQL] First phase: Deferring the correlated predicate pull up to Optimizer phase
## What changes were proposed in this pull request?
Currently Analyzer as part of ResolveSubquery, pulls up the correlated predicates to its
originating SubqueryExpression. The subquery plan is then transformed to remove the correlated
predicates after they are moved up to the outer plan. In this PR, the task of pulling up
correlated predicates is deferred to Optimizer. This is the initial work that will allow us to
support the form of correlated subqueries that we don't support today. The design document
from nsyca can be found in the following link :
[DesignDoc](https://docs.google.com/document/d/1QDZ8JwU63RwGFS6KVF54Rjj9ZJyK33d49ZWbjFBaIgU/edit#)

The brief description of code changes (hopefully to aid with code review) can be be found in the
following link:
[CodeChanges](https://docs.google.com/document/d/18mqjhL9V1An-tNta7aVE13HkALRZ5GZ24AATA-Vqqf0/edit#)

## How was this patch tested?
The test case PRs were submitted earlier using.
[16337](https://github.com/apache/spark/pull/16337) [16759](https://github.com/apache/spark/pull/16759) [16841](https://github.com/apache/spark/pull/16841) [16915](https://github.com/apache/spark/pull/16915) [16798](https://github.com/apache/spark/pull/16798) [16712](https://github.com/apache/spark/pull/16712) [16710](https://github.com/apache/spark/pull/16710) [16760](https://github.com/apache/spark/pull/16760) [16802](https://github.com/apache/spark/pull/16802)

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

Closes #16954 from dilipbiswal/SPARK-18874.
2017-03-14 10:37:10 +01:00
Tejas Patil 9456688547 [SPARK-17495][SQL] Support date, timestamp and interval types in Hive hash
## What changes were proposed in this pull request?

- Timestamp hashing is done as per [TimestampWritable.hashCode()](ff67cdda1c/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritable.java (L406)) in Hive
- Interval hashing is done as per [HiveIntervalDayTime.hashCode()](ff67cdda1c/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveIntervalDayTime.java (L178)). Note that there are inherent differences in how Hive and Spark store intervals under the hood which limits the ability to be in completely sync with hive's hashing function. I have explained this in the method doc.
- Date type was already supported. This PR adds test for that.

## How was this patch tested?

Added unit tests

Author: Tejas Patil <tejasp@fb.com>

Closes #17062 from tejasapatil/SPARK-17495_time_related_types.
2017-03-12 20:08:44 -07:00
Budde f79371ad86 [SPARK-19611][SQL] Introduce configurable table schema inference
## Summary of changes

Add a new configuration option that allows Spark SQL to infer a case-sensitive schema from a Hive Metastore table's data files when a case-sensitive schema can't be read from the table properties.

- Add spark.sql.hive.caseSensitiveInferenceMode param to SQLConf
- Add schemaPreservesCase field to CatalogTable (set to false when schema can't
  successfully be read from Hive table props)
- Perform schema inference in HiveMetastoreCatalog if schemaPreservesCase is
  false, depending on spark.sql.hive.caseSensitiveInferenceMode
- Add alterTableSchema() method to the ExternalCatalog interface
- Add HiveSchemaInferenceSuite tests
- Refactor and move ParquetFileForamt.meregeMetastoreParquetSchema() as
  HiveMetastoreCatalog.mergeWithMetastoreSchema
- Move schema merging tests from ParquetSchemaSuite to HiveSchemaInferenceSuite

[JIRA for this change](https://issues.apache.org/jira/browse/SPARK-19611)

## How was this patch tested?

The tests in ```HiveSchemaInferenceSuite``` should verify that schema inference is working as expected. ```ExternalCatalogSuite``` has also been extended to cover the new ```alterTableSchema()``` API.

Author: Budde <budde@amazon.com>

Closes #16944 from budde/SPARK-19611.
2017-03-09 12:55:33 -08:00
Kunal Khamar 6570cfd7ab [SPARK-19540][SQL] Add ability to clone SparkSession wherein cloned session has an identical copy of the SessionState
Forking a newSession() from SparkSession currently makes a new SparkSession that does not retain SessionState (i.e. temporary tables, SQL config, registered functions etc.) This change adds a method cloneSession() which creates a new SparkSession with a copy of the parent's SessionState.

Subsequent changes to base session are not propagated to cloned session, clone is independent after creation.
If the base is changed after clone has been created, say user registers new UDF, then the new UDF will not be available inside the clone. Same goes for configs and temp tables.

Unit tests

Author: Kunal Khamar <kkhamar@outlook.com>
Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16826 from kunalkhamar/fork-sparksession.
2017-03-08 13:20:45 -08:00
Shixiong Zhu 1bf9012380 [SPARK-19858][SS] Add output mode to flatMapGroupsWithState and disallow invalid cases
## What changes were proposed in this pull request?

Add a output mode parameter to `flatMapGroupsWithState` and just define `mapGroupsWithState` as `flatMapGroupsWithState(Update)`.

`UnsupportedOperationChecker` is modified to disallow unsupported cases.

- Batch mapGroupsWithState or flatMapGroupsWithState is always allowed.
- For streaming (map/flatMap)GroupsWithState, see the following table:

| Operators  | Supported Query Output Mode |
| ------------- | ------------- |
| flatMapGroupsWithState(Update) without aggregation  | Update |
| flatMapGroupsWithState(Update) with aggregation  | None |
| flatMapGroupsWithState(Append) without aggregation  | Append |
| flatMapGroupsWithState(Append) before aggregation  | Append, Update, Complete |
| flatMapGroupsWithState(Append) after aggregation  | None |
| Multiple flatMapGroupsWithState(Append)s  | Append |
| Multiple mapGroupsWithStates  | None |
| Mxing mapGroupsWithStates  and flatMapGroupsWithStates | None |
| Other cases of multiple flatMapGroupsWithState | None |

## How was this patch tested?

The added unit tests. Here are the tests related to (map/flatMap)GroupsWithState:
```
[info] - batch plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on batch relation: supported (1 millisecond)
[info] - batch plan - flatMapGroupsWithState - multiple flatMapGroupsWithState(Append)s on batch relation: supported (0 milliseconds)
[info] - batch plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on batch relation: supported (0 milliseconds)
[info] - batch plan - flatMapGroupsWithState - multiple flatMapGroupsWithState(Update)s on batch relation: supported (0 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation without aggregation in update mode: supported (2 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation without aggregation in append mode: not supported (7 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation without aggregation in complete mode: not supported (5 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation with aggregation in Append mode: not supported (11 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation with aggregation in Update mode: not supported (5 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation with aggregation in Complete mode: not supported (5 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation without aggregation in append mode: supported (1 millisecond)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation without aggregation in update mode: not supported (6 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation before aggregation in Append mode: supported (1 millisecond)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation before aggregation in Update mode: supported (0 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation before aggregation in Complete mode: supported (1 millisecond)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation after aggregation in Append mode: not supported (6 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation after aggregation in Update mode: not supported (4 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation in complete mode: not supported (2 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on batch relation inside streaming relation in Append output mode: supported (1 millisecond)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on batch relation inside streaming relation in Update output mode: supported (1 millisecond)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on batch relation inside streaming relation in Append output mode: supported (0 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on batch relation inside streaming relation in Update output mode: supported (0 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - multiple flatMapGroupsWithStates on streaming relation and all are in append mode: supported (2 milliseconds)
[info] - streaming plan - flatMapGroupsWithState -  multiple flatMapGroupsWithStates on s streaming relation but some are not in append mode: not supported (7 milliseconds)
[info] - streaming plan - mapGroupsWithState - mapGroupsWithState on streaming relation without aggregation in append mode: not supported (3 milliseconds)
[info] - streaming plan - mapGroupsWithState - mapGroupsWithState on streaming relation without aggregation in complete mode: not supported (3 milliseconds)
[info] - streaming plan - mapGroupsWithState - mapGroupsWithState on streaming relation with aggregation in Append mode: not supported (6 milliseconds)
[info] - streaming plan - mapGroupsWithState - mapGroupsWithState on streaming relation with aggregation in Update mode: not supported (3 milliseconds)
[info] - streaming plan - mapGroupsWithState - mapGroupsWithState on streaming relation with aggregation in Complete mode: not supported (4 milliseconds)
[info] - streaming plan - mapGroupsWithState - multiple mapGroupsWithStates on streaming relation and all are in append mode: not supported (4 milliseconds)
[info] - streaming plan - mapGroupsWithState - mixing mapGroupsWithStates and flatMapGroupsWithStates on streaming relation: not supported (4 milliseconds)
```

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #17197 from zsxwing/mapgroups-check.
2017-03-08 13:18:07 -08:00
Wojtek Szymanski e9e2c612d5 [SPARK-19727][SQL] Fix for round function that modifies original column
## What changes were proposed in this pull request?

Fix for SQL round function that modifies original column when underlying data frame is created from a local product.

    import org.apache.spark.sql.functions._

    case class NumericRow(value: BigDecimal)

    val df = spark.createDataFrame(Seq(NumericRow(BigDecimal("1.23456789"))))

    df.show()
    +--------------------+
    |               value|
    +--------------------+
    |1.234567890000000000|
    +--------------------+

    df.withColumn("value_rounded", round('value)).show()

    // before
    +--------------------+-------------+
    |               value|value_rounded|
    +--------------------+-------------+
    |1.000000000000000000|            1|
    +--------------------+-------------+

    // after
    +--------------------+-------------+
    |               value|value_rounded|
    +--------------------+-------------+
    |1.234567890000000000|            1|
    +--------------------+-------------+

## How was this patch tested?

New unit test added to existing suite `org.apache.spark.sql.MathFunctionsSuite`

Author: Wojtek Szymanski <wk.szymanski@gmail.com>

Closes #17075 from wojtek-szymanski/SPARK-19727.
2017-03-08 12:36:16 -08:00
Xiao Li 9a6ac7226f [SPARK-19601][SQL] Fix CollapseRepartition rule to preserve shuffle-enabled Repartition
### What changes were proposed in this pull request?

Observed by felixcheung  in https://github.com/apache/spark/pull/16739, when users use the shuffle-enabled `repartition` API, they expect the partition they got should be the exact number they provided, even if they call shuffle-disabled `coalesce` later.

Currently, `CollapseRepartition` rule does not consider whether shuffle is enabled or not. Thus, we got the following unexpected result.

```Scala
    val df = spark.range(0, 10000, 1, 5)
    val df2 = df.repartition(10)
    assert(df2.coalesce(13).rdd.getNumPartitions == 5)
    assert(df2.coalesce(7).rdd.getNumPartitions == 5)
    assert(df2.coalesce(3).rdd.getNumPartitions == 3)
```

This PR is to fix the issue. We preserve shuffle-enabled Repartition.

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #16933 from gatorsmile/CollapseRepartition.
2017-03-08 09:36:01 -08:00
jiangxingbo 5f7d835d38 [SPARK-19865][SQL] remove the view identifier in SubqueryAlias
## What changes were proposed in this pull request?

Since we have a `View` node now, we can remove the view identifier in `SubqueryAlias`, which was used to indicate a view node before.

## How was this patch tested?

Update the related test cases.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #17210 from jiangxb1987/SubqueryAlias.
2017-03-08 16:18:17 +01:00
wangzhenhua e44274870d [SPARK-17080][SQL] join reorder
## What changes were proposed in this pull request?

Reorder the joins using a dynamic programming algorithm (Selinger paper):
First we put all items (basic joined nodes) into level 1, then we build all two-way joins at level 2 from plans at level 1 (single items), then build all 3-way joins from plans at previous levels (two-way joins and single items), then 4-way joins ... etc, until we build all n-way joins and pick the best plan among them.

When building m-way joins, we only keep the best plan (with the lowest cost) for the same set of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among plans (A J B) J C, (A J C) J B and (B J C) J A. Thus, the plans maintained for each level when reordering four items A, B, C, D are as follows:
```
level 1: p({A}), p({B}), p({C}), p({D})
level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
level 4: p({A, B, C, D})
```
where p({A, B, C, D}) is the final output plan.

For cost evaluation, since physical costs for operators are not available currently, we use cardinalities and sizes to compute costs.

## How was this patch tested?
add test cases

Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>

Closes #17138 from wzhfy/joinReorder.
2017-03-08 16:01:28 +01:00
wangzhenhua 932196d9e3 [SPARK-17075][SQL][FOLLOWUP] fix filter estimation issues
## What changes were proposed in this pull request?

1. support boolean type in binary expression estimation.
2. deal with compound Not conditions.
3. avoid convert BigInt/BigDecimal directly to double unless it's within range (0, 1).
4. reorganize test code.

## How was this patch tested?

modify related test cases.

Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>

Closes #17148 from wzhfy/fixFilter.
2017-03-06 23:53:53 -08:00
wangzhenhua 9909f6d361 [SPARK-19350][SQL] Cardinality estimation of Limit and Sample
## What changes were proposed in this pull request?

Before this pr, LocalLimit/GlobalLimit/Sample propagates the same row count and column stats from its child, which is incorrect.
We can get the correct rowCount in Statistics for GlobalLimit/Sample whether cbo is enabled or not.
We don't know the rowCount for LocalLimit because we don't know the partition number at that time. Column stats should not be propagated because we don't know the distribution of columns after Limit or Sample.

## How was this patch tested?

Added test cases.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #16696 from wzhfy/limitEstimation.
2017-03-06 21:45:36 -08:00
windpiger 096df6d933 [SPARK-19257][SQL] location for table/partition/database should be java.net.URI
## What changes were proposed in this pull request?

Currently we treat the location of table/partition/database as URI string.

It will be safer if we can make the type of location as java.net.URI.

In this PR, there are following classes changes:
**1. CatalogDatabase**
```
case class CatalogDatabase(
    name: String,
    description: String,
    locationUri: String,
    properties: Map[String, String])
--->
case class CatalogDatabase(
    name: String,
    description: String,
    locationUri: URI,
    properties: Map[String, String])
```
**2. CatalogStorageFormat**
```
case class CatalogStorageFormat(
    locationUri: Option[String],
    inputFormat: Option[String],
    outputFormat: Option[String],
    serde: Option[String],
    compressed: Boolean,
    properties: Map[String, String])
---->
case class CatalogStorageFormat(
    locationUri: Option[URI],
    inputFormat: Option[String],
    outputFormat: Option[String],
    serde: Option[String],
    compressed: Boolean,
    properties: Map[String, String])
```

Before and After this PR, it is transparent for user, there is no change that the user should concern. The `String` to `URI` just happened in SparkSQL internally.

Here list some operation related location:
**1. whitespace in the location**
   e.g.  `/a/b c/d`
   For both table location and partition location,
   After `CREATE TABLE  t... (PARTITIONED BY ...) LOCATION '/a/b c/d'` ,
   then `DESC EXTENDED t ` show the location is `/a/b c/d`,
   and the real path in the FileSystem also show `/a/b c/d`

**2. colon(:) in the location**
   e.g.  `/a/b:c/d`
   For both table location and partition location,
   when `CREATE TABLE  t... (PARTITIONED BY ...)  LOCATION '/a/b:c/d'` ,

  **In linux file system**
   `DESC EXTENDED t ` show the location is `/a/b:c/d`,
   and the real path in the FileSystem also show `/a/b:c/d`

  **in HDFS** throw exception:
  `java.lang.IllegalArgumentException: Pathname /a/b:c/d from hdfs://iZbp1151s8hbnnwriekxdeZ:9000/a/b:c/d is not a valid DFS filename.`

  **while** After `INSERT INTO TABLE t PARTITION(a="a:b") SELECT 1`
   then `DESC EXTENDED t ` show the location is `/xxx/a=a%3Ab`,
   and the real path in the FileSystem also show `/xxx/a=a%3Ab`

**3. percent sign(%) in the location**
   e.g.  `/a/b%c/d`
   For both table location and partition location,
   After `CREATE TABLE  t... (PARTITIONED BY ...) LOCATION '/a/b%c/d'` ,
   then `DESC EXTENDED t ` show the location is `/a/b%c/d`,
   and the real path in the FileSystem also show `/a/b%c/d`

**4. encoded(%25) in the location**
   e.g.  `/a/b%25c/d`
   For both table location and partition location,
   After `CREATE TABLE  t... (PARTITIONED BY ...)  LOCATION '/a/b%25c/d'` ,
   then `DESC EXTENDED t ` show the location is `/a/b%25c/d`,
   and the real path in the FileSystem also show `/a/b%25c/d`

   **while** After `INSERT INTO TABLE t PARTITION(a="%25") SELECT 1`
   then `DESC EXTENDED t ` show the location is `/xxx/a=%2525`,
   and the real path in the FileSystem also show `/xxx/a=%2525`

**Additionally**, except the location, there are two other factors will affect the location of the table/partition. one is the table name which does not allowed to have special characters, and the  other is `partition name` which have the same actions with `partition value`, and `partition name` with special character situation has add some testcase and resolve a bug in [PR](https://github.com/apache/spark/pull/17173)

### Summary:
After `CREATE TABLE  t... (PARTITIONED BY ...)  LOCATION path`,
the path which we get from `DESC TABLE` and `real path in FileSystem` are all the same with the `CREATE TABLE` command(different filesystem has different action that allow what kind of special character to create the path, e.g. HDFS does not allow colon, but linux filesystem allow it ).

`DataBase` also have the same logic with `CREATE TABLE`

while if the `partition value` has some special character like `%` `:` `#` etc, then we will get the path with encoded `partition value` like `/xxx/a=A%25B` from `DESC TABLE` and `real path in FileSystem`

In this PR, the core change code is using `new Path(str).toUri` and `new Path(uri).toString`
which transfrom `str to uri `or `uri to str`.
for example:
```
val str = '/a/b c/d'
val uri = new Path(str).toUri  --> '/a/b%20c/d'
val strFromUri = new Path(uri).toString -> '/a/b c/d'
```

when we restore table/partition from metastore, or get the location from `CREATE TABLE` command, we can use it as above to change string to uri `new Path(str).toUri `

## How was this patch tested?
unit test added.
The `current master branch` also `passed all the test cases` added in this PR by a litter change.
https://github.com/apache/spark/pull/17149/files#diff-b7094baa12601424a5d19cb930e3402fR1764
here `toURI` -> `toString` when test in master branch.

This can show that this PR  is transparent for user.

Author: windpiger <songjun@outlook.com>

Closes #17149 from windpiger/changeStringToURI.
2017-03-06 10:44:26 -08:00
Cheng Lian 339b53a131 [SPARK-19737][SQL] New analysis rule for reporting unregistered functions without relying on relation resolution
## What changes were proposed in this pull request?

This PR adds a new `Once` analysis rule batch consists of a single analysis rule `LookupFunctions` that performs simple existence check over `UnresolvedFunctions` without actually resolving them.

The benefit of this rule is that it doesn't require function arguments to be resolved first and therefore doesn't rely on relation resolution, which may incur potentially expensive partition/schema discovery cost.

Please refer to [SPARK-19737][1] for more details about the motivation.

## How was this patch tested?

New test case added in `AnalysisErrorSuite`.

[1]: https://issues.apache.org/jira/browse/SPARK-19737

Author: Cheng Lian <lian@databricks.com>

Closes #17168 from liancheng/spark-19737-lookup-functions.
2017-03-06 10:36:50 -08:00
Tejas Patil 2a0bc867a4 [SPARK-17495][SQL] Support Decimal type in Hive-hash
## What changes were proposed in this pull request?

Hive hash to support Decimal datatype. [Hive internally normalises decimals](4ba713ccd8/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimalV1.java (L307)) and I have ported that logic as-is to HiveHash.

## How was this patch tested?

Added unit tests

Author: Tejas Patil <tejasp@fb.com>

Closes #17056 from tejasapatil/SPARK-17495_decimal.
2017-03-06 10:16:20 -08:00
hyukjinkwon 369a148e59 [SPARK-19595][SQL] Support json array in from_json
## What changes were proposed in this pull request?

This PR proposes to both,

**Do not allow json arrays with multiple elements and return null in `from_json` with `StructType` as the schema.**

Currently, it only reads the single row when the input is a json array. So, the codes below:

```scala
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._
val schema = StructType(StructField("a", IntegerType) :: Nil)
Seq(("""[{"a": 1}, {"a": 2}]""")).toDF("struct").select(from_json(col("struct"), schema)).show()
```
prints

```
+--------------------+
|jsontostruct(struct)|
+--------------------+
|                 [1]|
+--------------------+
```

This PR simply suggests to print this as `null` if the schema is `StructType` and input is json array.with multiple elements

```
+--------------------+
|jsontostruct(struct)|
+--------------------+
|                null|
+--------------------+
```

**Support json arrays in `from_json` with `ArrayType` as the schema.**

```scala
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._
val schema = ArrayType(StructType(StructField("a", IntegerType) :: Nil))
Seq(("""[{"a": 1}, {"a": 2}]""")).toDF("array").select(from_json(col("array"), schema)).show()
```

prints

```
+-------------------+
|jsontostruct(array)|
+-------------------+
|         [[1], [2]]|
+-------------------+
```

## How was this patch tested?

Unit test in `JsonExpressionsSuite`, `JsonFunctionsSuite`, Python doctests and manual test.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16929 from HyukjinKwon/disallow-array.
2017-03-05 14:35:06 -08:00
Takeshi Yamamuro 14bb398fae [SPARK-19254][SQL] Support Seq, Map, and Struct in functions.lit
## What changes were proposed in this pull request?
This pr is to support Seq, Map, and Struct in functions.lit; it adds a new IF named `lit2` with `TypeTag` for avoiding type erasure.

## How was this patch tested?
Added tests in `LiteralExpressionSuite`

Author: Takeshi Yamamuro <yamamuro@apache.org>
Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes #16610 from maropu/SPARK-19254.
2017-03-05 03:53:19 -08:00
Liang-Chi Hsieh 98bcc188f9 [SPARK-19758][SQL] Resolving timezone aware expressions with time zone when resolving inline table
## What changes were proposed in this pull request?

When we resolve inline tables in analyzer, we will evaluate the expressions of inline tables.

When it evaluates a `TimeZoneAwareExpression` expression, an error will happen because the `TimeZoneAwareExpression` is not associated with timezone yet.

So we need to resolve these `TimeZoneAwareExpression`s with time zone when resolving inline tables.

## How was this patch tested?

Jenkins tests.

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

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

Closes #17114 from viirya/resolve-timeawareexpr-inline-table.
2017-03-03 07:14:37 -08:00
Stan Zhai 5502a9cf88 [SPARK-19766][SQL] Constant alias columns in INNER JOIN should not be folded by FoldablePropagation rule
## What changes were proposed in this pull request?
This PR fixes the code in Optimizer phase where the constant alias columns of a `INNER JOIN` query are folded in Rule `FoldablePropagation`.

For the following query():

```
val sqlA =
  """
    |create temporary view ta as
    |select a, 'a' as tag from t1 union all
    |select a, 'b' as tag from t2
  """.stripMargin

val sqlB =
  """
    |create temporary view tb as
    |select a, 'a' as tag from t3 union all
    |select a, 'b' as tag from t4
  """.stripMargin

val sql =
  """
    |select tb.* from ta inner join tb on
    |ta.a = tb.a and
    |ta.tag = tb.tag
  """.stripMargin
```

The tag column is an constant alias column, it's folded by `FoldablePropagation` like this:

```
TRACE SparkOptimizer:
=== Applying Rule org.apache.spark.sql.catalyst.optimizer.FoldablePropagation ===
 Project [a#4, tag#14]                              Project [a#4, tag#14]
!+- Join Inner, ((a#0 = a#4) && (tag#8 = tag#14))   +- Join Inner, ((a#0 = a#4) && (a = a))
    :- Union                                           :- Union
    :  :- Project [a#0, a AS tag#8]                    :  :- Project [a#0, a AS tag#8]
    :  :  +- LocalRelation [a#0]                       :  :  +- LocalRelation [a#0]
    :  +- Project [a#2, b AS tag#9]                    :  +- Project [a#2, b AS tag#9]
    :     +- LocalRelation [a#2]                       :     +- LocalRelation [a#2]
    +- Union                                           +- Union
       :- Project [a#4, a AS tag#14]                      :- Project [a#4, a AS tag#14]
       :  +- LocalRelation [a#4]                          :  +- LocalRelation [a#4]
       +- Project [a#6, b AS tag#15]                      +- Project [a#6, b AS tag#15]
          +- LocalRelation [a#6]                             +- LocalRelation [a#6]
```

Finally the Result of Batch Operator Optimizations is:

```
Project [a#4, tag#14]                              Project [a#4, tag#14]
!+- Join Inner, ((a#0 = a#4) && (tag#8 = tag#14))   +- Join Inner, (a#0 = a#4)
!   :- SubqueryAlias ta, `ta`                          :- Union
!   :  +- Union                                        :  :- LocalRelation [a#0]
!   :     :- Project [a#0, a AS tag#8]                 :  +- LocalRelation [a#2]
!   :     :  +- SubqueryAlias t1, `t1`                 +- Union
!   :     :     +- Project [a#0]                          :- LocalRelation [a#4, tag#14]
!   :     :        +- SubqueryAlias grouping              +- LocalRelation [a#6, tag#15]
!   :     :           +- LocalRelation [a#0]
!   :     +- Project [a#2, b AS tag#9]
!   :        +- SubqueryAlias t2, `t2`
!   :           +- Project [a#2]
!   :              +- SubqueryAlias grouping
!   :                 +- LocalRelation [a#2]
!   +- SubqueryAlias tb, `tb`
!      +- Union
!         :- Project [a#4, a AS tag#14]
!         :  +- SubqueryAlias t3, `t3`
!         :     +- Project [a#4]
!         :        +- SubqueryAlias grouping
!         :           +- LocalRelation [a#4]
!         +- Project [a#6, b AS tag#15]
!            +- SubqueryAlias t4, `t4`
!               +- Project [a#6]
!                  +- SubqueryAlias grouping
!                     +- LocalRelation [a#6]
```

The condition `tag#8 = tag#14` of INNER JOIN has been removed. This leads to the data of inner join being wrong.

After fix:

```
=== Result of Batch LocalRelation ===
 GlobalLimit 21                                           GlobalLimit 21
 +- LocalLimit 21                                         +- LocalLimit 21
    +- Project [a#4, tag#11]                                 +- Project [a#4, tag#11]
       +- Join Inner, ((a#0 = a#4) && (tag#8 = tag#11))         +- Join Inner, ((a#0 = a#4) && (tag#8 = tag#11))
!         :- SubqueryAlias ta                                      :- Union
!         :  +- Union                                              :  :- LocalRelation [a#0, tag#8]
!         :     :- Project [a#0, a AS tag#8]                       :  +- LocalRelation [a#2, tag#9]
!         :     :  +- SubqueryAlias t1                             +- Union
!         :     :     +- Project [a#0]                                :- LocalRelation [a#4, tag#11]
!         :     :        +- SubqueryAlias grouping                    +- LocalRelation [a#6, tag#12]
!         :     :           +- LocalRelation [a#0]
!         :     +- Project [a#2, b AS tag#9]
!         :        +- SubqueryAlias t2
!         :           +- Project [a#2]
!         :              +- SubqueryAlias grouping
!         :                 +- LocalRelation [a#2]
!         +- SubqueryAlias tb
!            +- Union
!               :- Project [a#4, a AS tag#11]
!               :  +- SubqueryAlias t3
!               :     +- Project [a#4]
!               :        +- SubqueryAlias grouping
!               :           +- LocalRelation [a#4]
!               +- Project [a#6, b AS tag#12]
!                  +- SubqueryAlias t4
!                     +- Project [a#6]
!                        +- SubqueryAlias grouping
!                           +- LocalRelation [a#6]
```

## How was this patch tested?

add sql-tests/inputs/inner-join.sql
All tests passed.

Author: Stan Zhai <zhaishidan@haizhi.com>

Closes #17099 from stanzhai/fix-inner-join.
2017-03-01 07:52:35 -08:00
Wenchen Fan 7c7fc30b4a [SPARK-19678][SQL] remove MetastoreRelation
## What changes were proposed in this pull request?

`MetastoreRelation` is used to represent table relation for hive tables, and provides some hive related information. We will resolve `SimpleCatalogRelation` to `MetastoreRelation` for hive tables, which is unnecessary as these 2 are the same essentially. This PR merges `SimpleCatalogRelation` and `MetastoreRelation`

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17015 from cloud-fan/table-relation.
2017-02-28 09:24:36 -08:00
Wenchen Fan 89608cf262 [SPARK-17075][SQL][FOLLOWUP] fix some minor issues and clean up the code
## What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/16395. It fixes some code style issues, naming issues, some missing cases in pattern match, etc.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17065 from cloud-fan/follow-up.
2017-02-25 23:01:44 -08:00
wangzhenhua 69d0da6373 [SPARK-17078][SQL] Show stats when explain
## What changes were proposed in this pull request?

Currently we can only check the estimated stats in logical plans by debugging. We need to provide an easier and more efficient way for developers/users.

In this pr, we add EXPLAIN COST command to show stats in the optimized logical plan.
E.g.
```
spark-sql> EXPLAIN COST select count(1) from store_returns;

...
== Optimized Logical Plan ==
Aggregate [count(1) AS count(1)#24L], Statistics(sizeInBytes=16.0 B, rowCount=1, isBroadcastable=false)
+- Project, Statistics(sizeInBytes=4.3 GB, rowCount=5.76E+8, isBroadcastable=false)
   +- Relation[sr_returned_date_sk#3,sr_return_time_sk#4,sr_item_sk#5,sr_customer_sk#6,sr_cdemo_sk#7,sr_hdemo_sk#8,sr_addr_sk#9,sr_store_sk#10,sr_reason_sk#11,sr_ticket_number#12,sr_return_quantity#13,sr_return_amt#14,sr_return_tax#15,sr_return_amt_inc_tax#16,sr_fee#17,sr_return_ship_cost#18,sr_refunded_cash#19,sr_reversed_charge#20,sr_store_credit#21,sr_net_loss#22] parquet, Statistics(sizeInBytes=28.6 GB, rowCount=5.76E+8, isBroadcastable=false)
...
```

## How was this patch tested?

Add test cases.

Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>

Closes #16594 from wzhfy/showStats.
2017-02-24 10:24:59 -08:00
Shuai Lin 05954f32e9 [SPARK-17075][SQL] Follow up: fix file line ending and improve the tests
## What changes were proposed in this pull request?

Fixed the line ending of `FilterEstimation.scala` (It's still using `\n\r`). Also improved the tests to cover the cases where the literals are on the left side of a binary operator.

## How was this patch tested?

Existing unit tests.

Author: Shuai Lin <linshuai2012@gmail.com>

Closes #17051 from lins05/fix-cbo-filter-file-encoding.
2017-02-24 10:24:01 -08:00
Tejas Patil 3e40f6c3d6 [SPARK-17495][SQL] Add more tests for hive hash
## What changes were proposed in this pull request?

This PR adds tests hive-hash by comparing the outputs generated against Hive 1.2.1. Following datatypes are covered by this PR:
- null
- boolean
- byte
- short
- int
- long
- float
- double
- string
- array
- map
- struct

Datatypes that I have _NOT_ covered but I will work on separately are:
- Decimal (handled separately in https://github.com/apache/spark/pull/17056)
- TimestampType
- DateType
- CalendarIntervalType

## How was this patch tested?

NA

Author: Tejas Patil <tejasp@fb.com>

Closes #17049 from tejasapatil/SPARK-17495_remaining_types.
2017-02-24 09:46:42 -08:00
Ron Hu d7e43b613a [SPARK-17075][SQL] implemented filter estimation
## What changes were proposed in this pull request?

We traverse predicate and evaluate the logical expressions to compute the selectivity of a FILTER operator.

## How was this patch tested?

We add a new test suite to test various logical operators.

Author: Ron Hu <ron.hu@huawei.com>

Closes #16395 from ron8hu/filterSelectivity.
2017-02-23 20:18:21 -08:00
Shixiong Zhu 9bf4e2baad [SPARK-19497][SS] Implement streaming deduplication
## What changes were proposed in this pull request?

This PR adds a special streaming deduplication operator to support `dropDuplicates` with `aggregation` and watermark. It reuses the `dropDuplicates` API but creates new logical plan `Deduplication` and new physical plan `DeduplicationExec`.

The following cases are supported:

- one or multiple `dropDuplicates()` without aggregation (with or without watermark)
- `dropDuplicates` before aggregation

Not supported cases:

- `dropDuplicates` after aggregation

Breaking changes:
- `dropDuplicates` without aggregation doesn't work with `complete` or `update` mode.

## How was this patch tested?

The new unit tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16970 from zsxwing/dedup.
2017-02-23 11:25:39 -08:00
Takeshi Yamamuro 93aa427159 [SPARK-19691][SQL] Fix ClassCastException when calculating percentile of decimal column
## What changes were proposed in this pull request?
This pr fixed a class-cast exception below;
```
scala> spark.range(10).selectExpr("cast (id as decimal) as x").selectExpr("percentile(x, 0.5)").collect()
 java.lang.ClassCastException: org.apache.spark.sql.types.Decimal cannot be cast to java.lang.Number
	at org.apache.spark.sql.catalyst.expressions.aggregate.Percentile.update(Percentile.scala:141)
	at org.apache.spark.sql.catalyst.expressions.aggregate.Percentile.update(Percentile.scala:58)
	at org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggregate.update(interfaces.scala:514)
	at org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$1$$anonfun$applyOrElse$1.apply(AggregationIterator.scala:171)
	at org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$1$$anonfun$applyOrElse$1.apply(AggregationIterator.scala:171)
	at org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$generateProcessRow$1.apply(AggregationIterator.scala:187)
	at org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$generateProcessRow$1.apply(AggregationIterator.scala:181)
	at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.processInputs(ObjectAggregationIterator.scala:151)
	at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.<init>(ObjectAggregationIterator.scala:78)
	at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec$$anonfun$doExecute$1$$anonfun$2.apply(ObjectHashAggregateExec.scala:109)
	at
```
This fix simply converts catalyst values (i.e., `Decimal`) into scala ones by using `CatalystTypeConverters`.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17028 from maropu/SPARK-19691.
2017-02-23 16:28:36 +01:00
Xiao Li dc005ed53c [SPARK-19658][SQL] Set NumPartitions of RepartitionByExpression In Parser
### What changes were proposed in this pull request?

Currently, if `NumPartitions` is not set in RepartitionByExpression, we will set it using `spark.sql.shuffle.partitions` during Planner. However, this is not following the general resolution process. This PR is to set it in `Parser` and then `Optimizer` can use the value for plan optimization.

### How was this patch tested?

Added a test case.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #16988 from gatorsmile/resolveRepartition.
2017-02-22 17:26:56 -08:00
windpiger 65fe902e13 [SPARK-19598][SQL] Remove the alias parameter in UnresolvedRelation
## What changes were proposed in this pull request?

Remove the alias parameter in `UnresolvedRelation`, and use `SubqueryAlias` to replace it.
This can simplify some `match case` situations.

For example, the broadcast hint pull request can have one fewer case https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala#L57-L61

## How was this patch tested?
add some unit tests

Author: windpiger <songjun@outlook.com>

Closes #16956 from windpiger/removeUnresolveTableAlias.
2017-02-19 16:50:16 -08:00
Takuya UESHIN 865b2fd84c [SPARK-18937][SQL] Timezone support in CSV/JSON parsing
## What changes were proposed in this pull request?

This is a follow-up pr of #16308.

This pr enables timezone support in CSV/JSON parsing.

We should introduce `timeZone` option for CSV/JSON datasources (the default value of the option is session local timezone).

The datasources should use the `timeZone` option to format/parse to write/read timestamp values.
Notice that while reading, if the timestampFormat has the timezone info, the timezone will not be used because we should respect the timezone in the values.

For example, if you have timestamp `"2016-01-01 00:00:00"` in `GMT`, the values written with the default timezone option, which is `"GMT"` because session local timezone is `"GMT"` here, are:

```scala
scala> spark.conf.set("spark.sql.session.timeZone", "GMT")

scala> val df = Seq(new java.sql.Timestamp(1451606400000L)).toDF("ts")
df: org.apache.spark.sql.DataFrame = [ts: timestamp]

scala> df.show()
+-------------------+
|ts                 |
+-------------------+
|2016-01-01 00:00:00|
+-------------------+

scala> df.write.json("/path/to/gmtjson")
```

```sh
$ cat /path/to/gmtjson/part-*
{"ts":"2016-01-01T00:00:00.000Z"}
```

whereas setting the option to `"PST"`, they are:

```scala
scala> df.write.option("timeZone", "PST").json("/path/to/pstjson")
```

```sh
$ cat /path/to/pstjson/part-*
{"ts":"2015-12-31T16:00:00.000-08:00"}
```

We can properly read these files even if the timezone option is wrong because the timestamp values have timezone info:

```scala
scala> val schema = new StructType().add("ts", TimestampType)
schema: org.apache.spark.sql.types.StructType = StructType(StructField(ts,TimestampType,true))

scala> spark.read.schema(schema).json("/path/to/gmtjson").show()
+-------------------+
|ts                 |
+-------------------+
|2016-01-01 00:00:00|
+-------------------+

scala> spark.read.schema(schema).option("timeZone", "PST").json("/path/to/gmtjson").show()
+-------------------+
|ts                 |
+-------------------+
|2016-01-01 00:00:00|
+-------------------+
```

And even if `timezoneFormat` doesn't contain timezone info, we can properly read the values with setting correct timezone option:

```scala
scala> df.write.option("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss").option("timeZone", "JST").json("/path/to/jstjson")
```

```sh
$ cat /path/to/jstjson/part-*
{"ts":"2016-01-01T09:00:00"}
```

```scala
// wrong result
scala> spark.read.schema(schema).option("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss").json("/path/to/jstjson").show()
+-------------------+
|ts                 |
+-------------------+
|2016-01-01 09:00:00|
+-------------------+

// correct result
scala> spark.read.schema(schema).option("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss").option("timeZone", "JST").json("/path/to/jstjson").show()
+-------------------+
|ts                 |
+-------------------+
|2016-01-01 00:00:00|
+-------------------+
```

This pr also makes `JsonToStruct` and `StructToJson` `TimeZoneAwareExpression` to be able to evaluate values with timezone option.

## How was this patch tested?

Existing tests and added some tests.

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

Closes #16750 from ueshin/issues/SPARK-18937.
2017-02-15 13:26:34 -08:00
Liang-Chi Hsieh acf71c63cd [SPARK-16475][SQL] broadcast hint for SQL queries - disallow space as the delimiter
## What changes were proposed in this pull request?

A follow-up to disallow space as the delimiter in broadcast hint.

## How was this patch tested?

Jenkins test.

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

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

Closes #16941 from viirya/disallow-space-delimiter.
2017-02-15 18:48:02 +01:00
Zhenhua Wang 601b9c3e68 [SPARK-17076][SQL] Cardinality estimation for join based on basic column statistics
## What changes were proposed in this pull request?

Support cardinality estimation and stats propagation for all join types.

Limitations:
- For inner/outer joins without any equal condition, we estimate it like cartesian product.
- For left semi/anti joins, since we can't apply the heuristics for inner join to it, for now we just propagate the statistics from left side. We should support them when other advanced stats (e.g. histograms) are available in spark.

## How was this patch tested?

Add a new test suite.

Author: Zhenhua Wang <wzh_zju@163.com>
Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #16228 from wzhfy/joinEstimate.
2017-02-15 08:21:51 -08:00
Reynold Xin 733c59ec1e [SPARK-16475][SQL] broadcast hint for SQL queries - follow up
## What changes were proposed in this pull request?
A small update to https://github.com/apache/spark/pull/16925

1. Rename SubstituteHints -> ResolveHints to be more consistent with rest of the rules.
2. Added more documentation in the rule and be more defensive / future proof to skip views as well as CTEs.

## How was this patch tested?
This pull request contains no real logic change and all behavior should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #16939 from rxin/SPARK-16475.
2017-02-15 17:10:49 +01:00
Reynold Xin da7aef7a0e [SPARK-16475][SQL] Broadcast hint for SQL Queries
## What changes were proposed in this pull request?
This pull request introduces a simple hint infrastructure to SQL and implements broadcast join hint using the infrastructure.

The hint syntax looks like the following:
```
SELECT /*+ BROADCAST(t) */ * FROM t
```

For broadcast hint, we accept "BROADCAST", "BROADCASTJOIN", and "MAPJOIN", and a sequence of relation aliases can be specified in the hint. A broadcast hint plan node will be inserted on top of any relation (that is not aliased differently), subquery, or common table expression that match the specified name.

The hint resolution works by recursively traversing down the query plan to find a relation or subquery that matches one of the specified broadcast aliases. The traversal does not go past beyond any existing broadcast hints, subquery aliases. This rule happens before common table expressions.

Note that there was an earlier patch in https://github.com/apache/spark/pull/14426. This is a rewrite of that patch, with different semantics and simpler test cases.

## How was this patch tested?
Added a new unit test suite for the broadcast hint rule (SubstituteHintsSuite) and new test cases for parser change (in PlanParserSuite). Also added end-to-end test case in BroadcastSuite.

Author: Reynold Xin <rxin@databricks.com>
Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #16925 from rxin/SPARK-16475-broadcast-hint.
2017-02-14 14:11:17 -08:00
hyukjinkwon 9af8f743b0 [SPARK-19435][SQL] Type coercion between ArrayTypes
## What changes were proposed in this pull request?

This PR proposes to support type coercion between `ArrayType`s where the element types are compatible.

**Before**

```
Seq(Array(1)).toDF("a").selectExpr("greatest(a, array(1D))")
org.apache.spark.sql.AnalysisException: cannot resolve 'greatest(`a`, array(1.0D))' due to data type mismatch: The expressions should all have the same type, got GREATEST(array<int>, array<double>).; line 1 pos 0;

Seq(Array(1)).toDF("a").selectExpr("least(a, array(1D))")
org.apache.spark.sql.AnalysisException: cannot resolve 'least(`a`, array(1.0D))' due to data type mismatch: The expressions should all have the same type, got LEAST(array<int>, array<double>).; line 1 pos 0;

sql("SELECT * FROM values (array(0)), (array(1D)) as data(a)")
org.apache.spark.sql.AnalysisException: incompatible types found in column a for inline table; line 1 pos 14

Seq(Array(1)).toDF("a").union(Seq(Array(1D)).toDF("b"))
org.apache.spark.sql.AnalysisException: Union can only be performed on tables with the compatible column types. ArrayType(DoubleType,false) <> ArrayType(IntegerType,false) at the first column of the second table;;

sql("SELECT IF(1=1, array(1), array(1D))")
org.apache.spark.sql.AnalysisException: cannot resolve '(IF((1 = 1), array(1), array(1.0D)))' due to data type mismatch: differing types in '(IF((1 = 1), array(1), array(1.0D)))' (array<int> and array<double>).; line 1 pos 7;
```

**After**

```scala
Seq(Array(1)).toDF("a").selectExpr("greatest(a, array(1D))")
res5: org.apache.spark.sql.DataFrame = [greatest(a, array(1.0)): array<double>]

Seq(Array(1)).toDF("a").selectExpr("least(a, array(1D))")
res6: org.apache.spark.sql.DataFrame = [least(a, array(1.0)): array<double>]

sql("SELECT * FROM values (array(0)), (array(1D)) as data(a)")
res8: org.apache.spark.sql.DataFrame = [a: array<double>]

Seq(Array(1)).toDF("a").union(Seq(Array(1D)).toDF("b"))
res10: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [a: array<double>]

sql("SELECT IF(1=1, array(1), array(1D))")
res15: org.apache.spark.sql.DataFrame = [(IF((1 = 1), array(1), array(1.0))): array<double>]
```

## How was this patch tested?

Unit tests in `TypeCoercion` and Jenkins tests and

building with scala 2.10

```scala
./dev/change-scala-version.sh 2.10
./build/mvn -Pyarn -Phadoop-2.4 -Dscala-2.10 -DskipTests clean package
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16777 from HyukjinKwon/SPARK-19435.
2017-02-13 13:10:57 -08:00
hyukjinkwon 4321ff9edd [SPARK-19544][SQL] Improve error message when some column types are compatible and others are not in set operations
## What changes were proposed in this pull request?

This PR proposes to fix the error message when some data types are compatible and others are not in set/union operation.

Currently, the code below:

```scala
Seq((1,("a", 1))).toDF.union(Seq((1L,("a", "b"))).toDF)
```

throws an exception saying `LongType` and `IntegerType` are incompatible types. It should say something about `StructType`s with more readable format as below:

**Before**

```
Union can only be performed on tables with the compatible column types.
LongType <> IntegerType at the first column of the second table;;
```

**After**

```
Union can only be performed on tables with the compatible column types.
struct<_1:string,_2:string> <> struct<_1:string,_2:int> at the second column of the second table;;
```

*I manually inserted a newline in the messages above for readability only in this PR description.

## How was this patch tested?

Unit tests in `AnalysisErrorSuite`, manual tests and build wth Scala 2.10.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16882 from HyukjinKwon/SPARK-19544.
2017-02-13 16:08:31 +01:00
Burak Yavuz d5593f7f57 [SPARK-19543] from_json fails when the input row is empty
## What changes were proposed in this pull request?

Using from_json on a column with an empty string results in: java.util.NoSuchElementException: head of empty list.

This is because `parser.parse(input)` may return `Nil` when `input.trim.isEmpty`

## How was this patch tested?

Regression test in `JsonExpressionsSuite`

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #16881 from brkyvz/json-fix.
2017-02-10 12:55:06 +01:00
Tathagata Das aeb80348dd [SPARK-19413][SS] MapGroupsWithState for arbitrary stateful operations
## What changes were proposed in this pull request?

`mapGroupsWithState` is a new API for arbitrary stateful operations in Structured Streaming, similar to `DStream.mapWithState`

*Requirements*
- Users should be able to specify a function that can do the following
- Access the input row corresponding to a key
- Access the previous state corresponding to a key
- Optionally, update or remove the state
- Output any number of new rows (or none at all)

*Proposed API*
```
// ------------ New methods on KeyValueGroupedDataset ------------
class KeyValueGroupedDataset[K, V] {
	// Scala friendly
	def mapGroupsWithState[S: Encoder, U: Encoder](func: (K, Iterator[V], KeyedState[S]) => U)
        def flatMapGroupsWithState[S: Encode, U: Encoder](func: (K, Iterator[V], KeyedState[S]) => Iterator[U])
	// Java friendly
       def mapGroupsWithState[S, U](func: MapGroupsWithStateFunction[K, V, S, R], stateEncoder: Encoder[S], resultEncoder: Encoder[U])
       def flatMapGroupsWithState[S, U](func: FlatMapGroupsWithStateFunction[K, V, S, R], stateEncoder: Encoder[S], resultEncoder: Encoder[U])
}

// ------------------- New Java-friendly function classes -------------------
public interface MapGroupsWithStateFunction<K, V, S, R> extends Serializable {
  R call(K key, Iterator<V> values, state: KeyedState<S>) throws Exception;
}
public interface FlatMapGroupsWithStateFunction<K, V, S, R> extends Serializable {
  Iterator<R> call(K key, Iterator<V> values, state: KeyedState<S>) throws Exception;
}

// ---------------------- Wrapper class for state data ----------------------
trait State[S] {
	def exists(): Boolean
  	def get(): S 			// throws Exception is state does not exist
	def getOption(): Option[S]
	def update(newState: S): Unit
	def remove(): Unit		// exists() will be false after this
}
```

Key Semantics of the State class
- The state can be null.
- If the state.remove() is called, then state.exists() will return false, and getOption will returm None.
- After that state.update(newState) is called, then state.exists() will return true, and getOption will return Some(...).
- None of the operations are thread-safe. This is to avoid memory barriers.

*Usage*
```
val stateFunc = (word: String, words: Iterator[String, runningCount: KeyedState[Long]) => {
    val newCount = words.size + runningCount.getOption.getOrElse(0L)
    runningCount.update(newCount)
   (word, newCount)
}

dataset					                        // type is Dataset[String]
  .groupByKey[String](w => w)        	                // generates KeyValueGroupedDataset[String, String]
  .mapGroupsWithState[Long, (String, Long)](stateFunc)	// returns Dataset[(String, Long)]
```

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

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

Closes #16758 from tdas/mapWithState.
2017-02-07 20:21:00 -08:00
Herman van Hovell 73ee73945e [SPARK-18609][SPARK-18841][SQL] Fix redundant Alias removal in the optimizer
## What changes were proposed in this pull request?
The optimizer tries to remove redundant alias only projections from the query plan using the `RemoveAliasOnlyProject` rule. The current rule identifies removes such a project and rewrites the project's attributes in the **entire** tree. This causes problems when parts of the tree are duplicated (for instance a self join on a temporary view/CTE)  and the duplicated part contains the alias only project, in this case the rewrite will break the tree.

This PR fixes these problems by using a blacklist for attributes that are not to be moved, and by making sure that attribute remapping is only done for the parent tree, and not for unrelated parts of the query plan.

The current tree transformation infrastructure works very well if the transformation at hand requires little or a global contextual information. In this case we need to know both the attributes that were not to be moved, and we also needed to know which child attributes were modified. This cannot be done easily using the current infrastructure, and solutions typically involves transversing the query plan multiple times (which is super slow). I have moved around some code in `TreeNode`, `QueryPlan` and `LogicalPlan`to make this much more straightforward; this basically allows you to manually traverse the tree.

This PR subsumes the following PRs by windpiger:
Closes https://github.com/apache/spark/pull/16267
Closes https://github.com/apache/spark/pull/16255

## How was this patch tested?
I have added unit tests to `RemoveRedundantAliasAndProjectSuite` and I have added integration tests to the `SQLQueryTestSuite.union` and `SQLQueryTestSuite.cte` test cases.

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

Closes #16757 from hvanhovell/SPARK-18609.
2017-02-07 22:28:59 +01:00
gagan taneja e99e34d0f3 [SPARK-19118][SQL] Percentile support for frequency distribution table
## What changes were proposed in this pull request?

I have a frequency distribution table with following entries
Age,    No of person
21, 10
22, 15
23, 18
..
..
30, 14
Moreover it is common to have data in frequency distribution format to further calculate Percentile, Median. With current implementation
It would be very difficult and complex to find the percentile.
Therefore i am proposing enhancement to current Percentile and Approx Percentile implementation to take frequency distribution column into consideration

## How was this patch tested?
1) Enhanced /sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala to cover the additional functionality
2) Run some performance benchmark test with 20 million row in local environment and did not see any performance degradation

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

Author: gagan taneja <tanejagagan@gagans-MacBook-Pro.local>

Closes #16497 from tanejagagan/branch-18940.
2017-02-07 14:05:22 +01:00
Eyal Farago a97edc2cf4 [SPARK-18601][SQL] Simplify Create/Get complex expression pairs in optimizer
## What changes were proposed in this pull request?
It often happens that a complex object (struct/map/array) is created only to get elements from it in an subsequent expression. We can add an optimizer rule for this.

## How was this patch tested?
unit-tests

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

Author: Eyal Farago <eyal@nrgene.com>
Author: eyal farago <eyal.farago@gmail.com>

Closes #16043 from eyalfa/SPARK-18601.
2017-02-07 10:54:55 +01:00
Herman van Hovell cb2677b860 [SPARK-19472][SQL] Parser should not mistake CASE WHEN(...) for a function call
## What changes were proposed in this pull request?
The SQL parser can mistake a `WHEN (...)` used in `CASE` for a function call. This happens in cases like the following:
```sql
select case when (1) + case when 1 > 0 then 1 else 0 end = 2 then 1 else 0 end
from tb
```
This PR fixes this by re-organizing the case related parsing rules.

## How was this patch tested?
Added a regression test to the `ExpressionParserSuite`.

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

Closes #16821 from hvanhovell/SPARK-19472.
2017-02-06 15:28:13 -05:00
Liang-Chi Hsieh 0674e7eb85 [SPARK-19425][SQL] Make ExtractEquiJoinKeys support UDT columns
## What changes were proposed in this pull request?

DataFrame.except doesn't work for UDT columns. It is because `ExtractEquiJoinKeys` will run `Literal.default` against UDT. However, we don't handle UDT in `Literal.default` and an exception will throw like:

    java.lang.RuntimeException: no default for type
    org.apache.spark.ml.linalg.VectorUDT3bfc3ba7
      at org.apache.spark.sql.catalyst.expressions.Literal$.default(literals.scala:179)
      at org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys$$anonfun$4.apply(patterns.scala:117)
      at org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys$$anonfun$4.apply(patterns.scala:110)

More simple fix is just let `Literal.default` handle UDT by its sql type. So we can use more efficient join type on UDT.

Besides `except`, this also fixes other similar scenarios, so in summary this fixes:

* `except` on two Datasets with UDT
* `intersect` on two Datasets with UDT
* `Join` with the join conditions using `<=>` on UDT columns

## How was this patch tested?

Jenkins tests.

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

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

Closes #16765 from viirya/df-except-for-udt.
2017-02-04 15:57:56 -08:00
hyukjinkwon 2f3c20bbdd [SPARK-19446][SQL] Remove unused findTightestCommonType in TypeCoercion
## What changes were proposed in this pull request?

This PR proposes to

- remove unused `findTightestCommonType` in `TypeCoercion` as suggested in https://github.com/apache/spark/pull/16777#discussion_r99283834
- rename `findTightestCommonTypeOfTwo ` to `findTightestCommonType`.
- fix comments accordingly

The usage was removed while refactoring/fixing in several JIRAs such as SPARK-16714, SPARK-16735 and SPARK-16646

## How was this patch tested?

Existing tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16786 from HyukjinKwon/SPARK-19446.
2017-02-03 22:10:17 -08:00
Liang-Chi Hsieh bf493686eb [SPARK-19411][SQL] Remove the metadata used to mark optional columns in merged Parquet schema for filter predicate pushdown
## What changes were proposed in this pull request?

There is a metadata introduced before to mark the optional columns in merged Parquet schema for filter predicate pushdown. As we upgrade to Parquet 1.8.2 which includes the fix for the pushdown of optional columns, we don't need this metadata now.

## How was this patch tested?

Jenkins tests.

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

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

Closes #16756 from viirya/remove-optional-metadata.
2017-02-03 11:58:42 +01:00
Liwei Lin ade075aed4 [SPARK-19385][SQL] During canonicalization, NOT(...(l, r)) should not expect such cases that l.hashcode > r.hashcode
## What changes were proposed in this pull request?

During canonicalization, `NOT(...(l, r))` should not expect such cases that `l.hashcode > r.hashcode`.

Take the rule `case NOT(GreaterThan(l, r)) if l.hashcode > r.hashcode` for example, it should never be matched since `GreaterThan(l, r)` itself would be re-written as `GreaterThan(r, l)` given `l.hashcode > r.hashcode` after canonicalization.

This patch consolidates rules like `case NOT(GreaterThan(l, r)) if l.hashcode > r.hashcode` and `case NOT(GreaterThan(l, r))`.

## How was this patch tested?

This patch expanded the `NOT` test case to cover both cases where:
- `l.hashcode > r.hashcode`
- `l.hashcode < r.hashcode`

Author: Liwei Lin <lwlin7@gmail.com>

Closes #16719 from lw-lin/canonicalize.
2017-01-29 13:00:50 -08:00
Takuya UESHIN 2969fb4370 [SPARK-18936][SQL] Infrastructure for session local timezone support.
## What changes were proposed in this pull request?

As of Spark 2.1, Spark SQL assumes the machine timezone for datetime manipulation, which is bad if users are not in the same timezones as the machines, or if different users have different timezones.

We should introduce a session local timezone setting that is used for execution.

An explicit non-goal is locale handling.

### Semantics

Setting the session local timezone means that the timezone-aware expressions listed below should use the timezone to evaluate values, and also it should be used to convert (cast) between string and timestamp or between timestamp and date.

- `CurrentDate`
- `CurrentBatchTimestamp`
- `Hour`
- `Minute`
- `Second`
- `DateFormatClass`
- `ToUnixTimestamp`
- `UnixTimestamp`
- `FromUnixTime`

and below are implicitly timezone-aware through cast from timestamp to date:

- `DayOfYear`
- `Year`
- `Quarter`
- `Month`
- `DayOfMonth`
- `WeekOfYear`
- `LastDay`
- `NextDay`
- `TruncDate`

For example, if you have timestamp `"2016-01-01 00:00:00"` in `GMT`, the values evaluated by some of timezone-aware expressions are:

```scala
scala> val df = Seq(new java.sql.Timestamp(1451606400000L)).toDF("ts")
df: org.apache.spark.sql.DataFrame = [ts: timestamp]

scala> df.selectExpr("cast(ts as string)", "year(ts)", "month(ts)", "dayofmonth(ts)", "hour(ts)", "minute(ts)", "second(ts)").show(truncate = false)
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
|ts                 |year(CAST(ts AS DATE))|month(CAST(ts AS DATE))|dayofmonth(CAST(ts AS DATE))|hour(ts)|minute(ts)|second(ts)|
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
|2016-01-01 00:00:00|2016                  |1                      |1                           |0       |0         |0         |
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
```

whereas setting the session local timezone to `"PST"`, they are:

```scala
scala> spark.conf.set("spark.sql.session.timeZone", "PST")

scala> df.selectExpr("cast(ts as string)", "year(ts)", "month(ts)", "dayofmonth(ts)", "hour(ts)", "minute(ts)", "second(ts)").show(truncate = false)
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
|ts                 |year(CAST(ts AS DATE))|month(CAST(ts AS DATE))|dayofmonth(CAST(ts AS DATE))|hour(ts)|minute(ts)|second(ts)|
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
|2015-12-31 16:00:00|2015                  |12                     |31                          |16      |0         |0         |
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
```

Notice that even if you set the session local timezone, it affects only in `DataFrame` operations, neither in `Dataset` operations, `RDD` operations nor in `ScalaUDF`s. You need to properly handle timezone by yourself.

### Design of the fix

I introduced an analyzer to pass session local timezone to timezone-aware expressions and modified DateTimeUtils to take the timezone argument.

## How was this patch tested?

Existing tests and added tests for timezone aware expressions.

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

Closes #16308 from ueshin/issues/SPARK-18350.
2017-01-26 11:51:05 +01:00
Wenchen Fan 59c184e028 [SPARK-17913][SQL] compare atomic and string type column may return confusing result
## What changes were proposed in this pull request?

Spark SQL follows MySQL to do the implicit type conversion for binary comparison: http://dev.mysql.com/doc/refman/5.7/en/type-conversion.html

However, this may return confusing result, e.g. `1 = 'true'` will return true, `19157170390056973L = '19157170390056971'` will return true.

I think it's more reasonable to follow postgres in this case, i.e. cast string to the type of the other side, but return null if the string is not castable to keep hive compatibility.

## How was this patch tested?

newly added tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15880 from cloud-fan/compare.
2017-01-24 10:18:25 -08:00
jiangxingbo 3bdf3ee860 [SPARK-19272][SQL] Remove the param viewOriginalText from CatalogTable
## What changes were proposed in this pull request?

Hive will expand the view text, so it needs 2 fields: originalText and viewText. Since we don't expand the view text, but only add table properties, perhaps only a single field `viewText` is enough in CatalogTable.

This PR brought in the following changes:
1. Remove the param `viewOriginalText` from `CatalogTable`;
2. Update the output of command `DescribeTableCommand`.

## How was this patch tested?

Tested by exsiting test cases, also updated the failed test cases.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #16679 from jiangxb1987/catalogTable.
2017-01-24 12:37:30 +08:00
Wenchen Fan de6ad3dfa7 [SPARK-19309][SQL] disable common subexpression elimination for conditional expressions
## What changes were proposed in this pull request?

As I pointed out in https://github.com/apache/spark/pull/15807#issuecomment-259143655 , the current subexpression elimination framework has a problem, it always evaluates all common subexpressions at the beginning, even they are inside conditional expressions and may not be accessed.

Ideally we should implement it like scala lazy val, so we only evaluate it when it gets accessed at lease once. https://github.com/apache/spark/issues/15837 tries this approach, but it seems too complicated and may introduce performance regression.

This PR simply stops common subexpression elimination for conditional expressions, with some cleanup.

## How was this patch tested?

regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16659 from cloud-fan/codegen.
2017-01-23 13:31:26 +08:00
gatorsmile 772035e771 [SPARK-19229][SQL] Disallow Creating Hive Source Tables when Hive Support is Not Enabled
### What changes were proposed in this pull request?
It is weird to create Hive source tables when using InMemoryCatalog. We are unable to operate it. This PR is to block users to create Hive source tables.

### How was this patch tested?
Fixed the test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16587 from gatorsmile/blockHiveTable.
2017-01-22 20:37:37 -08:00
Tathagata Das 552e5f0884 [SPARK-19314][SS][CATALYST] Do not allow sort before aggregation in Structured Streaming plan
## What changes were proposed in this pull request?

Sort in a streaming plan should be allowed only after a aggregation in complete mode. Currently it is incorrectly allowed when present anywhere in the plan. It gives unpredictable potentially incorrect results.

## How was this patch tested?
New test

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

Closes #16662 from tdas/SPARK-19314.
2017-01-20 14:04:51 -08:00
wangzhenhua 039ed9fe8a [SPARK-19271][SQL] Change non-cbo estimation of aggregate
## What changes were proposed in this pull request?

Change non-cbo estimation behavior of aggregate:
- If groupExpression is empty, we can know row count (=1) and the corresponding size;
- otherwise, estimation falls back to UnaryNode's computeStats method, which should not propagate rowCount and attributeStats in Statistics because they are not estimated in that method.

## How was this patch tested?

Added test case

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #16631 from wzhfy/aggNoCbo.
2017-01-19 22:18:47 -08:00
Wenchen Fan 2e62560024 [SPARK-19265][SQL] make table relation cache general and does not depend on hive
## What changes were proposed in this pull request?

We have a table relation plan cache in `HiveMetastoreCatalog`, which caches a lot of things: file status, resolved data source, inferred schema, etc.

However, it doesn't make sense to limit this cache with hive support, we should move it to SQL core module so that users can use this cache without hive support.

It can also reduce the size of `HiveMetastoreCatalog`, so that it's easier to remove it eventually.

main changes:
1. move the table relation cache to `SessionCatalog`
2. `SessionCatalog.lookupRelation` will return `SimpleCatalogRelation` and the analyzer will convert it to `LogicalRelation` or `MetastoreRelation` later, then `HiveSessionCatalog` doesn't need to override `lookupRelation` anymore
3. `FindDataSourceTable` will read/write the table relation cache.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16621 from cloud-fan/plan-cache.
2017-01-19 00:07:48 -08:00
gatorsmile a23debd7bc [SPARK-19129][SQL] SessionCatalog: Disallow empty part col values in partition spec
### What changes were proposed in this pull request?
Empty partition column values are not valid for partition specification. Before this PR, we accept users to do it; however, Hive metastore does not detect and disallow it too. Thus, users hit the following strange error.

```Scala
val df = spark.createDataFrame(Seq((0, "a"), (1, "b"))).toDF("partCol1", "name")
df.write.mode("overwrite").partitionBy("partCol1").saveAsTable("partitionedTable")
spark.sql("alter table partitionedTable drop partition(partCol1='')")
spark.table("partitionedTable").show()
```

In the above example, the WHOLE table is DROPPED when users specify a partition spec containing only one partition column with empty values.

When the partition columns contains more than one, Hive metastore APIs simply ignore the columns with empty values and treat it as partial spec. This is also not expected. This does not follow the actual Hive behaviors. This PR is to disallow users to specify such an invalid partition spec in the `SessionCatalog` APIs.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16583 from gatorsmile/disallowEmptyPartColValue.
2017-01-18 02:01:30 +08:00
Wenchen Fan 871d266649 [SPARK-18969][SQL] Support grouping by nondeterministic expressions
## What changes were proposed in this pull request?

Currently nondeterministic expressions are allowed in `Aggregate`(see the [comment](https://github.com/apache/spark/blob/v2.0.2/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala#L249-L251)), but the `PullOutNondeterministic` analyzer rule failed to handle `Aggregate`, this PR fixes it.

close https://github.com/apache/spark/pull/16379

There is still one remaining issue: `SELECT a + rand() FROM t GROUP BY a + rand()` is not allowed, because the 2 `rand()` are different(we generate random seed as the default seed for `rand()`). https://issues.apache.org/jira/browse/SPARK-19035 is tracking this issue.

## How was this patch tested?

a new test suite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16404 from cloud-fan/groupby.
2017-01-12 20:21:04 +08:00
wangzhenhua 43fa21b3e6 [SPARK-19132][SQL] Add test cases for row size estimation and aggregate estimation
## What changes were proposed in this pull request?

In this pr, we add more test cases for project and aggregate estimation.

## How was this patch tested?

Add test cases.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #16551 from wzhfy/addTests.
2017-01-11 15:00:58 -08:00
jiangxingbo 30a07071f0 [SPARK-18801][SQL] Support resolve a nested view
## What changes were proposed in this pull request?

We should be able to resolve a nested view. The main advantage is that if you update an underlying view, the current view also gets updated.
The new approach should be compatible with older versions of SPARK/HIVE, that means:
1. The new approach should be able to resolve the views that created by older versions of SPARK/HIVE;
2. The new approach should be able to resolve the views that are currently supported by SPARK SQL.

The new approach mainly brings in the following changes:
1. Add a new operator called `View` to keep track of the CatalogTable that describes the view, and the output attributes as well as the child of the view;
2. Update the `ResolveRelations` rule to resolve the relations and views, note that a nested view should be resolved correctly;
3. Add `viewDefaultDatabase` variable to `CatalogTable` to keep track of the default database name used to resolve a view, if the `CatalogTable` is not a view, then the variable should be `None`;
4. Add `AnalysisContext` to enable us to still support a view created with CTE/Windows query;
5. Enables the view support without enabling Hive support (i.e., enableHiveSupport);
6. Fix a weird behavior: the result of a view query may have different schema if the referenced table has been changed. After this PR, we try to cast the child output attributes to that from the view schema, throw an AnalysisException if cast is not allowed.

Note this is compatible with the views defined by older versions of Spark(before 2.2), which have empty `defaultDatabase` and all the relations in `viewText` have database part defined.

## How was this patch tested?
1. Add new tests in `SessionCatalogSuite` to test the function `lookupRelation`;
2. Add new test case in `SQLViewSuite` to test resolve a nested view.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #16233 from jiangxb1987/resolve-view.
2017-01-11 13:44:07 -08:00
wangzhenhua a615513569 [SPARK-19149][SQL] Unify two sets of statistics in LogicalPlan
## What changes were proposed in this pull request?

Currently we have two sets of statistics in LogicalPlan: a simple stats and a stats estimated by cbo, but the computing logic and naming are quite confusing, we need to unify these two sets of stats.

## How was this patch tested?

Just modify existing tests.

Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>

Closes #16529 from wzhfy/unifyStats.
2017-01-10 22:34:44 -08:00
Shixiong Zhu bc6c56e940 [SPARK-19140][SS] Allow update mode for non-aggregation streaming queries
## What changes were proposed in this pull request?

This PR allow update mode for non-aggregation streaming queries. It will be same as the append mode if a query has no aggregations.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16520 from zsxwing/update-without-agg.
2017-01-10 17:58:11 -08:00
Liwei Lin acfc5f3543 [SPARK-16845][SQL] GeneratedClass$SpecificOrdering grows beyond 64 KB
## What changes were proposed in this pull request?

Prior to this patch, we'll generate `compare(...)` for `GeneratedClass$SpecificOrdering` like below, leading to Janino exceptions saying the code grows beyond 64 KB.

``` scala
/* 005 */ class SpecificOrdering extends o.a.s.sql.catalyst.expressions.codegen.BaseOrdering {
/* ..... */   ...
/* 10969 */   private int compare(InternalRow a, InternalRow b) {
/* 10970 */     InternalRow i = null;  // Holds current row being evaluated.
/* 10971 */
/* 1.... */     code for comparing field0
/* 1.... */     code for comparing field1
/* 1.... */     ...
/* 1.... */     code for comparing field449
/* 15012 */
/* 15013 */     return 0;
/* 15014 */   }
/* 15015 */ }
```

This patch would break `compare(...)` into smaller `compare_xxx(...)` methods when necessary; then we'll get generated `compare(...)` like:

``` scala
/* 001 */ public SpecificOrdering generate(Object[] references) {
/* 002 */   return new SpecificOrdering(references);
/* 003 */ }
/* 004 */
/* 005 */ class SpecificOrdering extends o.a.s.sql.catalyst.expressions.codegen.BaseOrdering {
/* 006 */
/* 007 */     ...
/* 1.... */
/* 11290 */   private int compare_0(InternalRow a, InternalRow b) {
/* 11291 */     InternalRow i = null;  // Holds current row being evaluated.
/* 11292 */
/* 11293 */     i = a;
/* 11294 */     boolean isNullA;
/* 11295 */     UTF8String primitiveA;
/* 11296 */     {
/* 11297 */
/* 11298 */       Object obj = ((Expression) references[0]).eval(null);
/* 11299 */       UTF8String value = (UTF8String) obj;
/* 11300 */       isNullA = false;
/* 11301 */       primitiveA = value;
/* 11302 */     }
/* 11303 */     i = b;
/* 11304 */     boolean isNullB;
/* 11305 */     UTF8String primitiveB;
/* 11306 */     {
/* 11307 */
/* 11308 */       Object obj = ((Expression) references[0]).eval(null);
/* 11309 */       UTF8String value = (UTF8String) obj;
/* 11310 */       isNullB = false;
/* 11311 */       primitiveB = value;
/* 11312 */     }
/* 11313 */     if (isNullA && isNullB) {
/* 11314 */       // Nothing
/* 11315 */     } else if (isNullA) {
/* 11316 */       return -1;
/* 11317 */     } else if (isNullB) {
/* 11318 */       return 1;
/* 11319 */     } else {
/* 11320 */       int comp = primitiveA.compare(primitiveB);
/* 11321 */       if (comp != 0) {
/* 11322 */         return comp;
/* 11323 */       }
/* 11324 */     }
/* 11325 */
/* 11326 */
/* 11327 */     i = a;
/* 11328 */     boolean isNullA1;
/* 11329 */     UTF8String primitiveA1;
/* 11330 */     {
/* 11331 */
/* 11332 */       Object obj1 = ((Expression) references[1]).eval(null);
/* 11333 */       UTF8String value1 = (UTF8String) obj1;
/* 11334 */       isNullA1 = false;
/* 11335 */       primitiveA1 = value1;
/* 11336 */     }
/* 11337 */     i = b;
/* 11338 */     boolean isNullB1;
/* 11339 */     UTF8String primitiveB1;
/* 11340 */     {
/* 11341 */
/* 11342 */       Object obj1 = ((Expression) references[1]).eval(null);
/* 11343 */       UTF8String value1 = (UTF8String) obj1;
/* 11344 */       isNullB1 = false;
/* 11345 */       primitiveB1 = value1;
/* 11346 */     }
/* 11347 */     if (isNullA1 && isNullB1) {
/* 11348 */       // Nothing
/* 11349 */     } else if (isNullA1) {
/* 11350 */       return -1;
/* 11351 */     } else if (isNullB1) {
/* 11352 */       return 1;
/* 11353 */     } else {
/* 11354 */       int comp = primitiveA1.compare(primitiveB1);
/* 11355 */       if (comp != 0) {
/* 11356 */         return comp;
/* 11357 */       }
/* 11358 */     }
/* 1.... */
/* 1.... */   ...
/* 1.... */
/* 12652 */     return 0;
/* 12653 */   }
/* 1.... */
/* 1.... */   ...
/* 15387 */
/* 15388 */   public int compare(InternalRow a, InternalRow b) {
/* 15389 */
/* 15390 */     int comp_0 = compare_0(a, b);
/* 15391 */     if (comp_0 != 0) {
/* 15392 */       return comp_0;
/* 15393 */     }
/* 15394 */
/* 15395 */     int comp_1 = compare_1(a, b);
/* 15396 */     if (comp_1 != 0) {
/* 15397 */       return comp_1;
/* 15398 */     }
/* 1.... */
/* 1.... */     ...
/* 1.... */
/* 15450 */     return 0;
/* 15451 */   }
/* 15452 */ }
```
## How was this patch tested?
- a new added test case which
  - would fail prior to this patch
  - would pass with this patch
- ordering correctness should already be covered by existing tests like those in `OrderingSuite`

## Acknowledgement

A major part of this PR - the refactoring work of `splitExpression()` - has been done by ueshin.

Author: Liwei Lin <lwlin7@gmail.com>
Author: Takuya UESHIN <ueshin@happy-camper.st>
Author: Takuya Ueshin <ueshin@happy-camper.st>

Closes #15480 from lw-lin/spec-ordering-64k-.
2017-01-10 19:35:46 +08:00
Zhenhua Wang 15c2bd01b0 [SPARK-19020][SQL] Cardinality estimation of aggregate operator
## What changes were proposed in this pull request?

Support cardinality estimation of aggregate operator

## How was this patch tested?

Add test cases

Author: Zhenhua Wang <wzh_zju@163.com>
Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #16431 from wzhfy/aggEstimation.
2017-01-09 11:29:42 -08:00
Zhenhua Wang 3ccabdfb4d [SPARK-17077][SQL] Cardinality estimation for project operator
## What changes were proposed in this pull request?

Support cardinality estimation for project operator.

## How was this patch tested?

Add a test suite and a base class in the catalyst package.

Author: Zhenhua Wang <wzh_zju@163.com>

Closes #16430 from wzhfy/projectEstimation.
2017-01-08 21:15:52 -08:00
Michal Senkyr 903bb8e8a2 [SPARK-16792][SQL] Dataset containing a Case Class with a List type causes a CompileException (converting sequence to list)
## What changes were proposed in this pull request?

Added a `to` call at the end of the code generated by `ScalaReflection.deserializerFor` if the requested type is not a supertype of `WrappedArray[_]` that uses `CanBuildFrom[_, _, _]` to convert result into an arbitrary subtype of `Seq[_]`.

Care was taken to preserve the original deserialization where it is possible to avoid the overhead of conversion in cases where it is not needed

`ScalaReflection.serializerFor` could already be used to serialize any `Seq[_]` so it was not altered

`SQLImplicits` had to be altered and new implicit encoders added to permit serialization of other sequence types

Also fixes [SPARK-16815] Dataset[List[T]] leads to ArrayStoreException

## How was this patch tested?
```bash
./build/mvn -DskipTests clean package && ./dev/run-tests
```

Also manual execution of the following sets of commands in the Spark shell:
```scala
case class TestCC(key: Int, letters: List[String])

val ds1 = sc.makeRDD(Seq(
(List("D")),
(List("S","H")),
(List("F","H")),
(List("D","L","L"))
)).map(x=>(x.length,x)).toDF("key","letters").as[TestCC]

val test1=ds1.map{_.key}
test1.show
```

```scala
case class X(l: List[String])
spark.createDataset(Seq(List("A"))).map(X).show
```

```scala
spark.sqlContext.createDataset(sc.parallelize(List(1) :: Nil)).collect
```

After adding arbitrary sequence support also tested with the following commands:

```scala
case class QueueClass(q: scala.collection.immutable.Queue[Int])

spark.createDataset(Seq(List(1,2,3))).map(x => QueueClass(scala.collection.immutable.Queue(x: _*))).map(_.q.dequeue).collect
```

Author: Michal Senkyr <mike.senkyr@gmail.com>

Closes #16240 from michalsenkyr/sql-caseclass-list-fix.
2017-01-06 15:05:20 +08:00
Niranjan Padmanabhan a1e40b1f5d
[MINOR][DOCS] Remove consecutive duplicated words/typo in Spark Repo
## What changes were proposed in this pull request?
There are many locations in the Spark repo where the same word occurs consecutively. Sometimes they are appropriately placed, but many times they are not. This PR removes the inappropriately duplicated words.

## How was this patch tested?
N/A since only docs or comments were updated.

Author: Niranjan Padmanabhan <niranjan.padmanabhan@gmail.com>

Closes #16455 from neurons/np.structure_streaming_doc.
2017-01-04 15:07:29 +00:00
Wenchen Fan cbd11d2357 [SPARK-19072][SQL] codegen of Literal should not output boxed value
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/16402 we made a mistake that, when double/float is infinity, the `Literal` codegen will output boxed value and cause wrong result.

This PR fixes this by special handling infinity to not output boxed value.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16469 from cloud-fan/literal.
2017-01-03 22:40:14 -08:00
gatorsmile b67b35f76b [SPARK-19048][SQL] Delete Partition Location when Dropping Managed Partitioned Tables in InMemoryCatalog
### What changes were proposed in this pull request?
The data in the managed table should be deleted after table is dropped. However, if the partition location is not under the location of the partitioned table, it is not deleted as expected. Users can specify any location for the partition when they adding a partition.

This PR is to delete partition location when dropping managed partitioned tables stored in `InMemoryCatalog`.

### How was this patch tested?
Added test cases for both HiveExternalCatalog and InMemoryCatalog

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16448 from gatorsmile/unsetSerdeProp.
2017-01-03 11:43:47 -08:00
Liang-Chi Hsieh 52636226dc [SPARK-18932][SQL] Support partial aggregation for collect_set/collect_list
## What changes were proposed in this pull request?

Currently collect_set/collect_list aggregation expression don't support partial aggregation. This patch is to enable partial aggregation for them.

## How was this patch tested?

Jenkins tests.

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

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

Closes #16371 from viirya/collect-partial-support.
2017-01-03 22:11:54 +08:00
Zhenhua Wang ae83c21125 [SPARK-18998][SQL] Add a cbo conf to switch between default statistics and estimated statistics
## What changes were proposed in this pull request?

We add a cbo configuration to switch between default stats and estimated stats.
We also define a new statistics method `planStats` in LogicalPlan with conf as its parameter, in order to pass the cbo switch and other estimation related configurations in the future. `planStats` is used on the caller sides (i.e. in Optimizer and Strategies) to make transformation decisions based on stats.

## How was this patch tested?

Add a test case using a dummy LogicalPlan.

Author: Zhenhua Wang <wzh_zju@163.com>

Closes #16401 from wzhfy/cboSwitch.
2017-01-03 12:19:52 +08:00
gatorsmile a6cd9dbc60 [SPARK-19029][SQL] Remove databaseName from SimpleCatalogRelation
### What changes were proposed in this pull request?
Remove useless `databaseName ` from `SimpleCatalogRelation`.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16438 from gatorsmile/removeDBFromSimpleCatalogRelation.
2017-01-03 11:55:31 +08:00
hyukjinkwon 852782b83c
[SPARK-18922][TESTS] Fix more path-related test failures on Windows
## What changes were proposed in this pull request?

This PR proposes to fix the test failures due to different format of paths on Windows.

Failed tests are as below:

```
ColumnExpressionSuite:
- input_file_name, input_file_block_start, input_file_block_length - FileScanRDD *** FAILED *** (187 milliseconds)
  "file:///C:/projects/spark/target/tmp/spark-0b21b963-6cfa-411c-8d6f-e6a5e1e73bce/part-00001-c083a03a-e55e-4b05-9073-451de352d006.snappy.parquet" did not contain "C:\projects\spark\target\tmp\spark-0b21b963-6cfa-411c-8d6f-e6a5e1e73bce" (ColumnExpressionSuite.scala:545)

- input_file_name, input_file_block_start, input_file_block_length - HadoopRDD *** FAILED *** (172 milliseconds)
  "file:/C:/projects/spark/target/tmp/spark-5d0afa94-7c2f-463b-9db9-2e8403e2bc5f/part-00000-f6530138-9ad3-466d-ab46-0eeb6f85ed0b.txt" did not contain "C:\projects\spark\target\tmp\spark-5d0afa94-7c2f-463b-9db9-2e8403e2bc5f" (ColumnExpressionSuite.scala:569)

- input_file_name, input_file_block_start, input_file_block_length - NewHadoopRDD *** FAILED *** (156 milliseconds)
  "file:/C:/projects/spark/target/tmp/spark-a894c7df-c74d-4d19-82a2-a04744cb3766/part-00000-29674e3f-3fcf-4327-9b04-4dab1d46338d.txt" did not contain "C:\projects\spark\target\tmp\spark-a894c7df-c74d-4d19-82a2-a04744cb3766" (ColumnExpressionSuite.scala:598)
```

```
DataStreamReaderWriterSuite:
- source metadataPath *** FAILED *** (62 milliseconds)
  org.mockito.exceptions.verification.junit.ArgumentsAreDifferent: Argument(s) are different! Wanted:
streamSourceProvider.createSource(
    org.apache.spark.sql.SQLContext3b04133b,
    "C:\projects\spark\target\tmp\streaming.metadata-b05db6ae-c8dc-4ce4-b0d9-1eb8c84876c0/sources/0",
    None,
    "org.apache.spark.sql.streaming.test",
    Map()
);
-> at org.apache.spark.sql.streaming.test.DataStreamReaderWriterSuite$$anonfun$12.apply$mcV$sp(DataStreamReaderWriterSuite.scala:374)
Actual invocation has different arguments:
streamSourceProvider.createSource(
    org.apache.spark.sql.SQLContext3b04133b,
    "/C:/projects/spark/target/tmp/streaming.metadata-b05db6ae-c8dc-4ce4-b0d9-1eb8c84876c0/sources/0",
    None,
    "org.apache.spark.sql.streaming.test",
    Map()
);
```

```
GlobalTempViewSuite:
- CREATE GLOBAL TEMP VIEW USING *** FAILED *** (110 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark  arget mpspark-960398ba-a0a1-45f6-a59a-d98533f9f519;
```

```
CreateTableAsSelectSuite:
- CREATE TABLE USING AS SELECT *** FAILED *** (0 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string

- create a table, drop it and create another one with the same name *** FAILED *** (16 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string

- create table using as select - with partitioned by *** FAILED *** (0 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string

- create table using as select - with non-zero buckets *** FAILED *** (0 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string
```

```
HiveMetadataCacheSuite:
- partitioned table is cached when partition pruning is true *** FAILED *** (532 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- partitioned table is cached when partition pruning is false *** FAILED *** (297 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
MultiDatabaseSuite:
- createExternalTable() to non-default database - with USE *** FAILED *** (954 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark  arget mpspark-0839d9a7-5e29-467a-9e3e-3e4cd618ee09;

- createExternalTable() to non-default database - without USE *** FAILED *** (500 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark  arget mpspark-c7e24d73-1d8f-45e8-ab7d-53a83087aec3;

 - invalid database name and table names *** FAILED *** (31 milliseconds)
   "Path does not exist: file:/C:projectsspark  arget mpspark-15a2a494-3483-4876-80e5-ec396e704b77;" did not contain "`t:a` is not a valid name for tables/databases. Valid names only contain alphabet characters, numbers and _." (MultiDatabaseSuite.scala:296)
```

```
OrcQuerySuite:
 - SPARK-8501: Avoids discovery schema from empty ORC files *** FAILED *** (15 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - Verify the ORC conversion parameter: CONVERT_METASTORE_ORC *** FAILED *** (78 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - converted ORC table supports resolving mixed case field *** FAILED *** (297 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
HadoopFsRelationTest - JsonHadoopFsRelationSuite, OrcHadoopFsRelationSuite, ParquetHadoopFsRelationSuite, SimpleTextHadoopFsRelationSuite:
 - Locality support for FileScanRDD *** FAILED *** (15 milliseconds)
   java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-383d1f13-8783-47fd-964d-9c75e5eec50f, expected: file:///
```

```
HiveQuerySuite:
- CREATE TEMPORARY FUNCTION *** FAILED *** (0 milliseconds)
   java.net.MalformedURLException: For input string: "%5Cprojects%5Cspark%5Csql%5Chive%5Ctarget%5Cscala-2.11%5Ctest-classes%5CTestUDTF.jar"

 - ADD FILE command *** FAILED *** (500 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\sql\hive\target\scala-2.11\test-classes\data\files\v1.txt

 - ADD JAR command 2 *** FAILED *** (110 milliseconds)
   org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive  argetscala-2.11 est-classesdatafilessample.json;
```

```
PruneFileSourcePartitionsSuite:
 - PruneFileSourcePartitions should not change the output of LogicalRelation *** FAILED *** (15 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
HiveCommandSuite:
 - LOAD DATA LOCAL *** FAILED *** (109 milliseconds)
   org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive  argetscala-2.11 est-classesdatafilesemployee.dat;

 - LOAD DATA *** FAILED *** (93 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 15: C:projectsspark arget mpemployee.dat7496657117354281006.tmp

 - Truncate Table *** FAILED *** (78 milliseconds)
   org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive  argetscala-2.11 est-classesdatafilesemployee.dat;
```

```
HiveExternalCatalogBackwardCompatibilitySuite:
- make sure we can read table created by old version of Spark *** FAILED *** (0 milliseconds)
  "[/C:/projects/spark/target/tmp/]spark-0554d859-74e1-..." did not equal "[C:\projects\spark\target\tmp\]spark-0554d859-74e1-..." (HiveExternalCatalogBackwardCompatibilitySuite.scala:213)
  org.scalatest.exceptions.TestFailedException

- make sure we can alter table location created by old version of Spark *** FAILED *** (110 milliseconds)
  java.net.URISyntaxException: Illegal character in opaque part at index 15: C:projectsspark	arget	mpspark-0e9b2c5f-49a1-4e38-a32a-c0ab1813a79f
```

```
ExternalCatalogSuite:
- create/drop/rename partitions should create/delete/rename the directory *** FAILED *** (610 milliseconds)
  java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-4c24f010-18df-437b-9fed-990c6f9adece
```

```
SQLQuerySuite:
- describe functions - temporary user defined functions *** FAILED *** (16 milliseconds)
  java.net.URISyntaxException: Illegal character in opaque part at index 22: C:projectssparksqlhive	argetscala-2.11	est-classesTestUDTF.jar

- specifying database name for a temporary table is not allowed *** FAILED *** (125 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-a34c9814-a483-43f2-be29-37f616b6df91;
```

```
PartitionProviderCompatibilitySuite:
- convert partition provider to hive with repair table *** FAILED *** (281 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-ee5fc96d-8c7d-4ebf-8571-a1d62736473e;

- when partition management is enabled, new tables have partition provider hive *** FAILED *** (187 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-803ad4d6-3e8c-498d-9ca5-5cda5d9b2a48;

- when partition management is disabled, new tables have no partition provider *** FAILED *** (172 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-c9fda9e2-4020-465f-8678-52cd72d0a58f;

- when partition management is disabled, we preserve the old behavior even for new tables *** FAILED *** (203 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget
mpspark-f4a518a6-c49d-43d3-b407-0ddd76948e13;

- insert overwrite partition of legacy datasource table *** FAILED *** (188 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-f4a518a6-c49d-43d3-b407-0ddd76948e79;

- insert overwrite partition of new datasource table overwrites just partition *** FAILED *** (219 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-6ba3a88d-6f6c-42c5-a9f4-6d924a0616ff;

- SPARK-18544 append with saveAsTable - partition management true *** FAILED *** (173 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-cd234a6d-9cb4-4d1d-9e51-854ae9543bbd;

- SPARK-18635 special chars in partition values - partition management true *** FAILED *** (2 seconds, 967 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- SPARK-18635 special chars in partition values - partition management false *** FAILED *** (62 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- SPARK-18659 insert overwrite table with lowercase - partition management true *** FAILED *** (63 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- SPARK-18544 append with saveAsTable - partition management false *** FAILED *** (266 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- SPARK-18659 insert overwrite table files - partition management false *** FAILED *** (63 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- SPARK-18659 insert overwrite table with lowercase - partition management false *** FAILED *** (78 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- sanity check table setup *** FAILED *** (31 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- insert into partial dynamic partitions *** FAILED *** (47 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- insert into fully dynamic partitions *** FAILED *** (62 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- insert into static partition *** FAILED *** (78 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- overwrite partial dynamic partitions *** FAILED *** (63 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- overwrite fully dynamic partitions *** FAILED *** (47 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- overwrite static partition *** FAILED *** (63 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
MetastoreDataSourcesSuite:
- check change without refresh *** FAILED *** (203 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-00713fe4-ca04-448c-bfc7-6c5e9a2ad2a1;

- drop, change, recreate *** FAILED *** (78 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-2030a21b-7d67-4385-a65b-bb5e2bed4861;

- SPARK-15269 external data source table creation *** FAILED *** (78 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-4d50fd4a-14bc-41d6-9232-9554dd233f86;

- CTAS *** FAILED *** (109 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string

- CTAS with IF NOT EXISTS *** FAILED *** (109 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string

- CTAS: persisted partitioned bucketed data source table *** FAILED *** (0 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string

- SPARK-15025: create datasource table with path with select *** FAILED *** (16 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string

- CTAS: persisted partitioned data source table *** FAILED *** (47 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string
```

```
HiveMetastoreCatalogSuite:
- Persist non-partitioned parquet relation into metastore as managed table using CTAS *** FAILED *** (16 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string

- Persist non-partitioned orc relation into metastore as managed table using CTAS *** FAILED *** (16 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string
```

```
HiveUDFSuite:
- SPARK-11522 select input_file_name from non-parquet table *** FAILED *** (16 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
QueryPartitionSuite:
- SPARK-13709: reading partitioned Avro table with nested schema *** FAILED *** (250 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
ParquetHiveCompatibilitySuite:
- simple primitives *** FAILED *** (16 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- SPARK-10177 timestamp *** FAILED *** (0 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- array *** FAILED *** (16 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- map *** FAILED *** (16 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- struct *** FAILED *** (0 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- SPARK-16344: array of struct with a single field named 'array_element' *** FAILED *** (15 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

## How was this patch tested?

Manually tested via AppVeyor.

```
ColumnExpressionSuite:
- input_file_name, input_file_block_start, input_file_block_length - FileScanRDD (234 milliseconds)
- input_file_name, input_file_block_start, input_file_block_length - HadoopRDD (235 milliseconds)
- input_file_name, input_file_block_start, input_file_block_length - NewHadoopRDD (203 milliseconds)
```

```
DataStreamReaderWriterSuite:
- source metadataPath (63 milliseconds)
```

```
GlobalTempViewSuite:
 - CREATE GLOBAL TEMP VIEW USING (436 milliseconds)
```

```
CreateTableAsSelectSuite:
- CREATE TABLE USING AS SELECT (171 milliseconds)
- create a table, drop it and create another one with the same name (422 milliseconds)
- create table using as select - with partitioned by (141 milliseconds)
- create table using as select - with non-zero buckets (125 milliseconds)
```

```
HiveMetadataCacheSuite:
- partitioned table is cached when partition pruning is true (3 seconds, 211 milliseconds)
- partitioned table is cached when partition pruning is false (1 second, 781 milliseconds)
```

```
MultiDatabaseSuite:
 - createExternalTable() to non-default database - with USE (797 milliseconds)
 - createExternalTable() to non-default database - without USE (640 milliseconds)
 - invalid database name and table names (62 milliseconds)
```

```
OrcQuerySuite:
 - SPARK-8501: Avoids discovery schema from empty ORC files (703 milliseconds)
 - Verify the ORC conversion parameter: CONVERT_METASTORE_ORC (750 milliseconds)
 - converted ORC table supports resolving mixed case field (625 milliseconds)
```

```
HadoopFsRelationTest - JsonHadoopFsRelationSuite, OrcHadoopFsRelationSuite, ParquetHadoopFsRelationSuite, SimpleTextHadoopFsRelationSuite:
 - Locality support for FileScanRDD (296 milliseconds)
```

```
HiveQuerySuite:
 - CREATE TEMPORARY FUNCTION (125 milliseconds)
 - ADD FILE command (250 milliseconds)
 - ADD JAR command 2 (609 milliseconds)
```

```
PruneFileSourcePartitionsSuite:
- PruneFileSourcePartitions should not change the output of LogicalRelation (359 milliseconds)
```

```
HiveCommandSuite:
 - LOAD DATA LOCAL (1 second, 829 milliseconds)
 - LOAD DATA (1 second, 735 milliseconds)
 - Truncate Table (1 second, 641 milliseconds)
```

```
HiveExternalCatalogBackwardCompatibilitySuite:
 - make sure we can read table created by old version of Spark (32 milliseconds)
 - make sure we can alter table location created by old version of Spark (125 milliseconds)
 - make sure we can rename table created by old version of Spark (281 milliseconds)
```

```
ExternalCatalogSuite:
- create/drop/rename partitions should create/delete/rename the directory (625 milliseconds)
```

```
SQLQuerySuite:
- describe functions - temporary user defined functions (31 milliseconds)
- specifying database name for a temporary table is not allowed (390 milliseconds)
```

```
PartitionProviderCompatibilitySuite:
 - convert partition provider to hive with repair table (813 milliseconds)
 - when partition management is enabled, new tables have partition provider hive (562 milliseconds)
 - when partition management is disabled, new tables have no partition provider (344 milliseconds)
 - when partition management is disabled, we preserve the old behavior even for new tables (422 milliseconds)
 - insert overwrite partition of legacy datasource table (750 milliseconds)
 - SPARK-18544 append with saveAsTable - partition management true (985 milliseconds)
 - SPARK-18635 special chars in partition values - partition management true (3 seconds, 328 milliseconds)
 - SPARK-18635 special chars in partition values - partition management false (2 seconds, 891 milliseconds)
 - SPARK-18659 insert overwrite table with lowercase - partition management true (750 milliseconds)
 - SPARK-18544 append with saveAsTable - partition management false (656 milliseconds)
 - SPARK-18659 insert overwrite table files - partition management false (922 milliseconds)
 - SPARK-18659 insert overwrite table with lowercase - partition management false (469 milliseconds)
 - sanity check table setup (937 milliseconds)
 - insert into partial dynamic partitions (2 seconds, 985 milliseconds)
 - insert into fully dynamic partitions (1 second, 937 milliseconds)
 - insert into static partition (1 second, 578 milliseconds)
 - overwrite partial dynamic partitions (7 seconds, 561 milliseconds)
 - overwrite fully dynamic partitions (1 second, 766 milliseconds)
 - overwrite static partition (1 second, 797 milliseconds)
```

```
MetastoreDataSourcesSuite:
 - check change without refresh (610 milliseconds)
 - drop, change, recreate (437 milliseconds)
 - SPARK-15269 external data source table creation (297 milliseconds)
 - CTAS with IF NOT EXISTS (437 milliseconds)
 - CTAS: persisted partitioned bucketed data source table (422 milliseconds)
 - SPARK-15025: create datasource table with path with select (265 milliseconds)
 - CTAS (438 milliseconds)
 - CTAS with IF NOT EXISTS (469 milliseconds)
 - CTAS: persisted partitioned bucketed data source table (406 milliseconds)
```

```
HiveMetastoreCatalogSuite:
 - Persist non-partitioned parquet relation into metastore as managed table using CTAS (406 milliseconds)
 - Persist non-partitioned orc relation into metastore as managed table using CTAS (313 milliseconds)
```

```
HiveUDFSuite:
 - SPARK-11522 select input_file_name from non-parquet table (3 seconds, 144 milliseconds)
```

```
QueryPartitionSuite:
 - SPARK-13709: reading partitioned Avro table with nested schema (1 second, 67 milliseconds)
```

```
ParquetHiveCompatibilitySuite:
 - simple primitives (745 milliseconds)
 - SPARK-10177 timestamp (375 milliseconds)
 - array (407 milliseconds)
 - map (409 milliseconds)
 - struct (437 milliseconds)
 - SPARK-16344: array of struct with a single field named 'array_element' (391 milliseconds)
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16397 from HyukjinKwon/SPARK-18922-paths.
2016-12-30 11:16:03 +00:00
Kazuaki Ishizaki 93f35569fd [SPARK-16213][SQL] Reduce runtime overhead of a program that creates an primitive array in DataFrame
## What changes were proposed in this pull request?

This PR reduces runtime overhead of a program the creates an primitive array in DataFrame by using the similar approach to #15044. Generated code performs boxing operation in an assignment from InternalRow to an `Object[]` temporary array (at Lines 051 and 061 in the generated code before without this PR). If we know that type of array elements is primitive, we apply the following optimizations:
1. Eliminate a pair of `isNullAt()` and a null assignment
2. Allocate an primitive array instead of `Object[]` (eliminate boxing operations)
3. Create `UnsafeArrayData` by using `UnsafeArrayWriter` to keep a primitive array in a row format instead of doing non-lightweight operations in constructor of `GenericArrayData`
The PR also performs the same things for `CreateMap`.

Here are performance results of [DataFrame programs](6bf54ec5e2/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala (L83-L112)) by up to 17.9x over without this PR.

```
Without SPARK-16043
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)
Read a primitive array in DataFrame:     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                           3805 / 4150          0.0      507308.9       1.0X
Double                                        3593 / 3852          0.0      479056.9       1.1X

With SPARK-16043
Read a primitive array in DataFrame:     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            213 /  271          0.0       28387.5       1.0X
Double                                         204 /  223          0.0       27250.9       1.0X
```
Note : #15780 is enabled for these measurements

An motivating example

``` java
val df = sparkContext.parallelize(Seq(0.0d, 1.0d), 1).toDF
df.selectExpr("Array(value + 1.1d, value + 2.2d)").show
```

Generated code without this PR

``` java
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private scala.collection.Iterator inputadapter_input;
/* 009 */   private UnsafeRow serializefromobject_result;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 012 */   private Object[] project_values;
/* 013 */   private UnsafeRow project_result;
/* 014 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder project_holder;
/* 015 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter project_rowWriter;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter project_arrayWriter;
/* 017 */
/* 018 */   public GeneratedIterator(Object[] references) {
/* 019 */     this.references = references;
/* 020 */   }
/* 021 */
/* 022 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 023 */     partitionIndex = index;
/* 024 */     this.inputs = inputs;
/* 025 */     inputadapter_input = inputs[0];
/* 026 */     serializefromobject_result = new UnsafeRow(1);
/* 027 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 0);
/* 028 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 029 */     this.project_values = null;
/* 030 */     project_result = new UnsafeRow(1);
/* 031 */     this.project_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(project_result, 32);
/* 032 */     this.project_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(project_holder, 1);
/* 033 */     this.project_arrayWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
/* 034 */
/* 035 */   }
/* 036 */
/* 037 */   protected void processNext() throws java.io.IOException {
/* 038 */     while (inputadapter_input.hasNext()) {
/* 039 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 040 */       double inputadapter_value = inputadapter_row.getDouble(0);
/* 041 */
/* 042 */       final boolean project_isNull = false;
/* 043 */       this.project_values = new Object[2];
/* 044 */       boolean project_isNull1 = false;
/* 045 */
/* 046 */       double project_value1 = -1.0;
/* 047 */       project_value1 = inputadapter_value + 1.1D;
/* 048 */       if (false) {
/* 049 */         project_values[0] = null;
/* 050 */       } else {
/* 051 */         project_values[0] = project_value1;
/* 052 */       }
/* 053 */
/* 054 */       boolean project_isNull4 = false;
/* 055 */
/* 056 */       double project_value4 = -1.0;
/* 057 */       project_value4 = inputadapter_value + 2.2D;
/* 058 */       if (false) {
/* 059 */         project_values[1] = null;
/* 060 */       } else {
/* 061 */         project_values[1] = project_value4;
/* 062 */       }
/* 063 */
/* 064 */       final ArrayData project_value = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_values);
/* 065 */       this.project_values = null;
/* 066 */       project_holder.reset();
/* 067 */
/* 068 */       project_rowWriter.zeroOutNullBytes();
/* 069 */
/* 070 */       if (project_isNull) {
/* 071 */         project_rowWriter.setNullAt(0);
/* 072 */       } else {
/* 073 */         // Remember the current cursor so that we can calculate how many bytes are
/* 074 */         // written later.
/* 075 */         final int project_tmpCursor = project_holder.cursor;
/* 076 */
/* 077 */         if (project_value instanceof UnsafeArrayData) {
/* 078 */           final int project_sizeInBytes = ((UnsafeArrayData) project_value).getSizeInBytes();
/* 079 */           // grow the global buffer before writing data.
/* 080 */           project_holder.grow(project_sizeInBytes);
/* 081 */           ((UnsafeArrayData) project_value).writeToMemory(project_holder.buffer, project_holder.cursor);
/* 082 */           project_holder.cursor += project_sizeInBytes;
/* 083 */
/* 084 */         } else {
/* 085 */           final int project_numElements = project_value.numElements();
/* 086 */           project_arrayWriter.initialize(project_holder, project_numElements, 8);
/* 087 */
/* 088 */           for (int project_index = 0; project_index < project_numElements; project_index++) {
/* 089 */             if (project_value.isNullAt(project_index)) {
/* 090 */               project_arrayWriter.setNullDouble(project_index);
/* 091 */             } else {
/* 092 */               final double project_element = project_value.getDouble(project_index);
/* 093 */               project_arrayWriter.write(project_index, project_element);
/* 094 */             }
/* 095 */           }
/* 096 */         }
/* 097 */
/* 098 */         project_rowWriter.setOffsetAndSize(0, project_tmpCursor, project_holder.cursor - project_tmpCursor);
/* 099 */       }
/* 100 */       project_result.setTotalSize(project_holder.totalSize());
/* 101 */       append(project_result);
/* 102 */       if (shouldStop()) return;
/* 103 */     }
/* 104 */   }
/* 105 */ }
```

Generated code with this PR

``` java
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private scala.collection.Iterator inputadapter_input;
/* 009 */   private UnsafeRow serializefromobject_result;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 012 */   private UnsafeArrayData project_arrayData;
/* 013 */   private UnsafeRow project_result;
/* 014 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder project_holder;
/* 015 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter project_rowWriter;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter project_arrayWriter;
/* 017 */
/* 018 */   public GeneratedIterator(Object[] references) {
/* 019 */     this.references = references;
/* 020 */   }
/* 021 */
/* 022 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 023 */     partitionIndex = index;
/* 024 */     this.inputs = inputs;
/* 025 */     inputadapter_input = inputs[0];
/* 026 */     serializefromobject_result = new UnsafeRow(1);
/* 027 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 0);
/* 028 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 029 */
/* 030 */     project_result = new UnsafeRow(1);
/* 031 */     this.project_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(project_result, 32);
/* 032 */     this.project_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(project_holder, 1);
/* 033 */     this.project_arrayWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
/* 034 */
/* 035 */   }
/* 036 */
/* 037 */   protected void processNext() throws java.io.IOException {
/* 038 */     while (inputadapter_input.hasNext()) {
/* 039 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 040 */       double inputadapter_value = inputadapter_row.getDouble(0);
/* 041 */
/* 042 */       byte[] project_array = new byte[32];
/* 043 */       project_arrayData = new UnsafeArrayData();
/* 044 */       Platform.putLong(project_array, 16, 2);
/* 045 */       project_arrayData.pointTo(project_array, 16, 32);
/* 046 */
/* 047 */       boolean project_isNull1 = false;
/* 048 */
/* 049 */       double project_value1 = -1.0;
/* 050 */       project_value1 = inputadapter_value + 1.1D;
/* 051 */       if (false) {
/* 052 */         project_arrayData.setNullAt(0);
/* 053 */       } else {
/* 054 */         project_arrayData.setDouble(0, project_value1);
/* 055 */       }
/* 056 */
/* 057 */       boolean project_isNull4 = false;
/* 058 */
/* 059 */       double project_value4 = -1.0;
/* 060 */       project_value4 = inputadapter_value + 2.2D;
/* 061 */       if (false) {
/* 062 */         project_arrayData.setNullAt(1);
/* 063 */       } else {
/* 064 */         project_arrayData.setDouble(1, project_value4);
/* 065 */       }
/* 066 */       project_holder.reset();
/* 067 */
/* 068 */       // Remember the current cursor so that we can calculate how many bytes are
/* 069 */       // written later.
/* 070 */       final int project_tmpCursor = project_holder.cursor;
/* 071 */
/* 072 */       if (project_arrayData instanceof UnsafeArrayData) {
/* 073 */         final int project_sizeInBytes = ((UnsafeArrayData) project_arrayData).getSizeInBytes();
/* 074 */         // grow the global buffer before writing data.
/* 075 */         project_holder.grow(project_sizeInBytes);
/* 076 */         ((UnsafeArrayData) project_arrayData).writeToMemory(project_holder.buffer, project_holder.cursor);
/* 077 */         project_holder.cursor += project_sizeInBytes;
/* 078 */
/* 079 */       } else {
/* 080 */         final int project_numElements = project_arrayData.numElements();
/* 081 */         project_arrayWriter.initialize(project_holder, project_numElements, 8);
/* 082 */
/* 083 */         for (int project_index = 0; project_index < project_numElements; project_index++) {
/* 084 */           if (project_arrayData.isNullAt(project_index)) {
/* 085 */             project_arrayWriter.setNullDouble(project_index);
/* 086 */           } else {
/* 087 */             final double project_element = project_arrayData.getDouble(project_index);
/* 088 */             project_arrayWriter.write(project_index, project_element);
/* 089 */           }
/* 090 */         }
/* 091 */       }
/* 092 */
/* 093 */       project_rowWriter.setOffsetAndSize(0, project_tmpCursor, project_holder.cursor - project_tmpCursor);
/* 094 */       project_result.setTotalSize(project_holder.totalSize());
/* 095 */       append(project_result);
/* 096 */       if (shouldStop()) return;
/* 097 */     }
/* 098 */   }
/* 099 */ }
```
## How was this patch tested?

Added unit tests into `DataFrameComplexTypeSuite`

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #13909 from kiszk/SPARK-16213.
2016-12-29 10:59:37 +08:00
Reynold Xin 2615100055 [SPARK-18973][SQL] Remove SortPartitions and RedistributeData
## What changes were proposed in this pull request?
SortPartitions and RedistributeData logical operators are not actually used and can be removed. Note that we do have a Sort operator (with global flag false) that subsumed SortPartitions.

## How was this patch tested?
Also updated test cases to reflect the removal.

Author: Reynold Xin <rxin@databricks.com>

Closes #16381 from rxin/SPARK-18973.
2016-12-22 19:35:09 +01:00
Tathagata Das 83a6ace0d1 [SPARK-18234][SS] Made update mode public
## What changes were proposed in this pull request?

Made update mode public. As part of that here are the changes.
- Update DatastreamWriter to accept "update"
- Changed package of InternalOutputModes from o.a.s.sql to o.a.s.sql.catalyst
- Added update mode state removing with watermark to StateStoreSaveExec

## How was this patch tested?

Added new tests in changed modules

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

Closes #16360 from tdas/SPARK-18234.
2016-12-21 16:43:17 -08:00
jiangxingbo 70d495dcec [SPARK-18624][SQL] Implicit cast ArrayType(InternalType)
## What changes were proposed in this pull request?

Currently `ImplicitTypeCasts` doesn't handle casts between `ArrayType`s, this is not convenient, we should add a rule to enable casting from `ArrayType(InternalType)` to `ArrayType(newInternalType)`.

Goals:
1. Add a rule to `ImplicitTypeCasts` to enable casting between `ArrayType`s;
2. Simplify `Percentile` and `ApproximatePercentile`.

## How was this patch tested?

Updated test cases in `TypeCoercionSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #16057 from jiangxb1987/implicit-cast-complex-types.
2016-12-19 21:20:47 +01:00
Tathagata Das 4f7292c875 [SPARK-18870] Disallowed Distinct Aggregations on Streaming Datasets
## What changes were proposed in this pull request?

Check whether Aggregation operators on a streaming subplan have aggregate expressions with isDistinct = true.

## How was this patch tested?

Added unit test

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

Closes #16289 from tdas/SPARK-18870.
2016-12-15 11:54:35 -08:00
Reynold Xin 5d79947369 [SPARK-18853][SQL] Project (UnaryNode) is way too aggressive in estimating statistics
## What changes were proposed in this pull request?
This patch reduces the default number element estimation for arrays and maps from 100 to 1. The issue with the 100 number is that when nested (e.g. an array of map), 100 * 100 would be used as the default size. This sounds like just an overestimation which doesn't seem that bad (since it is usually better to overestimate than underestimate). However, due to the way we assume the size output for Project (new estimated column size / old estimated column size), this overestimation can become underestimation. It is actually in general in this case safer to assume 1 default element.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #16274 from rxin/SPARK-18853.
2016-12-14 21:22:49 +01:00
Wenchen Fan 3e307b4959 [SPARK-18566][SQL] remove OverwriteOptions
## What changes were proposed in this pull request?

`OverwriteOptions` was introduced in https://github.com/apache/spark/pull/15705, to carry the information of static partitions. However, after further refactor, this information becomes duplicated and we can remove `OverwriteOptions`.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15995 from cloud-fan/overwrite.
2016-12-14 11:30:34 +08:00
Wenchen Fan 9abd05b6b9
[SQL][MINOR] simplify a test to fix the maven tests
## What changes were proposed in this pull request?

After https://github.com/apache/spark/pull/15620 , all of the Maven-based 2.0 Jenkins jobs time out consistently. As I pointed out in https://github.com/apache/spark/pull/15620#discussion_r91829129 , it seems that the regression test is an overkill and may hit constants pool size limitation, which is a known issue and hasn't been fixed yet.

Since #15620 only fix the code size limitation problem, we can simplify the test to avoid hitting constants pool size limitation.

## How was this patch tested?

test only change

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16244 from cloud-fan/minor.
2016-12-11 09:12:46 +00:00
Michael Allman 772ddbeaa6 [SPARK-18572][SQL] Add a method listPartitionNames to ExternalCatalog
(Link to Jira issue: https://issues.apache.org/jira/browse/SPARK-18572)

## What changes were proposed in this pull request?

Currently Spark answers the `SHOW PARTITIONS` command by fetching all of the table's partition metadata from the external catalog and constructing partition names therefrom. The Hive client has a `getPartitionNames` method which is many times faster for this purpose, with the performance improvement scaling with the number of partitions in a table.

To test the performance impact of this PR, I ran the `SHOW PARTITIONS` command on two Hive tables with large numbers of partitions. One table has ~17,800 partitions, and the other has ~95,000 partitions. For the purposes of this PR, I'll call the former table `table1` and the latter table `table2`. I ran 5 trials for each table with before-and-after versions of this PR. The results are as follows:

Spark at bdc8153, `SHOW PARTITIONS table1`, times in seconds:
7.901
3.983
4.018
4.331
4.261

Spark at bdc8153, `SHOW PARTITIONS table2`
(Timed out after 10 minutes with a `SocketTimeoutException`.)

Spark at this PR, `SHOW PARTITIONS table1`, times in seconds:
3.801
0.449
0.395
0.348
0.336

Spark at this PR, `SHOW PARTITIONS table2`, times in seconds:
5.184
1.63
1.474
1.519
1.41

Taking the best times from each trial, we get a 12x performance improvement for a table with ~17,800 partitions and at least a 426x improvement for a table with ~95,000 partitions. More significantly, the latter command doesn't even complete with the current code in master.

This is actually a patch we've been using in-house at VideoAmp since Spark 1.1. It's made all the difference in the practical usability of our largest tables. Even with tables with about 1,000 partitions there's a performance improvement of about 2-3x.

## How was this patch tested?

I added a unit test to `VersionsSuite` which tests that the Hive client's `getPartitionNames` method returns the correct number of partitions.

Author: Michael Allman <michael@videoamp.com>

Closes #15998 from mallman/spark-18572-list_partition_names.
2016-12-06 11:33:35 +08:00
Kapil Singh e463678b19 [SPARK-18091][SQL] Deep if expressions cause Generated SpecificUnsafeProjection code to exceed JVM code size limit
## What changes were proposed in this pull request?

Fix for SPARK-18091 which is a bug related to large if expressions causing generated SpecificUnsafeProjection code to exceed JVM code size limit.

This PR changes if expression's code generation to place its predicate, true value and false value expressions' generated code in separate methods in context so as to never generate too long combined code.
## How was this patch tested?

Added a unit test and also tested manually with the application (having transformations similar to the unit test) which caused the issue to be identified in the first place.

Author: Kapil Singh <kapsingh@adobe.com>

Closes #15620 from kapilsingh5050/SPARK-18091-IfCodegenFix.
2016-12-04 17:16:40 +08:00
Nattavut Sutyanyong 4a3c09601b [SPARK-18582][SQL] Whitelist LogicalPlan operators allowed in correlated subqueries
## What changes were proposed in this pull request?

This fix puts an explicit list of operators that Spark supports for correlated subqueries.

## How was this patch tested?

Run sql/test, catalyst/test and add a new test case on Generate.

Author: Nattavut Sutyanyong <nsy.can@gmail.com>

Closes #16046 from nsyca/spark18455.0.
2016-12-03 11:36:26 -08:00
Ryan Blue 48778976e0 [SPARK-18677] Fix parsing ['key'] in JSON path expressions.
## What changes were proposed in this pull request?

This fixes the parser rule to match named expressions, which doesn't work for two reasons:
1. The name match is not coerced to a regular expression (missing .r)
2. The surrounding literals are incorrect and attempt to escape a single quote, which is unnecessary

## How was this patch tested?

This adds test cases for named expressions using the bracket syntax, including one with quoted spaces.

Author: Ryan Blue <blue@apache.org>

Closes #16107 from rdblue/SPARK-18677-fix-json-path.
2016-12-02 08:41:40 -08:00
gatorsmile 2f8776ccad [SPARK-18674][SQL][FOLLOW-UP] improve the error message of using join
### What changes were proposed in this pull request?
Added a test case for using joins with nested fields.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16110 from gatorsmile/followup-18674.
2016-12-02 22:12:19 +08:00
Eric Liang 7935c8470c [SPARK-18659][SQL] Incorrect behaviors in overwrite table for datasource tables
## What changes were proposed in this pull request?

Two bugs are addressed here
1. INSERT OVERWRITE TABLE sometime crashed when catalog partition management was enabled. This was because when dropping partitions after an overwrite operation, the Hive client will attempt to delete the partition files. If the entire partition directory was dropped, this would fail. The PR fixes this by adding a flag to control whether the Hive client should attempt to delete files.
2. The static partition spec for OVERWRITE TABLE was not correctly resolved to the case-sensitive original partition names. This resulted in the entire table being overwritten if you did not correctly capitalize your partition names.

cc yhuai cloud-fan

## How was this patch tested?

Unit tests. Surprisingly, the existing overwrite table tests did not catch these edge cases.

Author: Eric Liang <ekl@databricks.com>

Closes #16088 from ericl/spark-18659.
2016-12-02 21:59:02 +08:00
Reynold Xin d3c90b74ed [SPARK-18663][SQL] Simplify CountMinSketch aggregate implementation
## What changes were proposed in this pull request?
SPARK-18429 introduced count-min sketch aggregate function for SQL, but the implementation and testing is more complicated than needed. This simplifies the test cases and removes support for data types that don't have clear equality semantics:

1. Removed support for floating point and decimal types.

2. Removed the heavy randomized tests. The underlying CountMinSketch implementation already had pretty good test coverage through randomized tests, and the SPARK-18429 implementation is just to add an aggregate function wrapper around CountMinSketch. There is no need for randomized tests at three different levels of the implementations.

## How was this patch tested?
A lot of the change is to simplify test cases.

Author: Reynold Xin <rxin@databricks.com>

Closes #16093 from rxin/SPARK-18663.
2016-12-01 21:38:52 -08:00
Kazuaki Ishizaki 38b9e69623 [SPARK-18284][SQL] Make ExpressionEncoder.serializer.nullable precise
## What changes were proposed in this pull request?

This PR makes `ExpressionEncoder.serializer.nullable` for flat encoder for a primitive type `false`. Since it is `true` for now, it is too conservative.
While `ExpressionEncoder.schema` has correct information (e.g. `<IntegerType, false>`), `serializer.head.nullable` of `ExpressionEncoder`, which got from `encoderFor[T]`, is always false. It is too conservative.

This is accomplished by checking whether a type is one of primitive types. If it is `true`, `nullable` should be `false`.

## How was this patch tested?

Added new tests for encoder and dataframe

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

Closes #15780 from kiszk/SPARK-18284.
2016-12-02 12:30:13 +08:00
Wenchen Fan e653484710 [SPARK-18674][SQL] improve the error message of using join
## What changes were proposed in this pull request?

The current error message of USING join is quite confusing, for example:
```
scala> val df1 = List(1,2,3).toDS.withColumnRenamed("value", "c1")
df1: org.apache.spark.sql.DataFrame = [c1: int]

scala> val df2 = List(1,2,3).toDS.withColumnRenamed("value", "c2")
df2: org.apache.spark.sql.DataFrame = [c2: int]

scala> df1.join(df2, usingColumn = "c1")
org.apache.spark.sql.AnalysisException: using columns ['c1] can not be resolved given input columns: [c1, c2] ;;
'Join UsingJoin(Inner,List('c1))
:- Project [value#1 AS c1#3]
:  +- LocalRelation [value#1]
+- Project [value#7 AS c2#9]
   +- LocalRelation [value#7]
```

after this PR, it becomes:
```
scala> val df1 = List(1,2,3).toDS.withColumnRenamed("value", "c1")
df1: org.apache.spark.sql.DataFrame = [c1: int]

scala> val df2 = List(1,2,3).toDS.withColumnRenamed("value", "c2")
df2: org.apache.spark.sql.DataFrame = [c2: int]

scala> df1.join(df2, usingColumn = "c1")
org.apache.spark.sql.AnalysisException: USING column `c1` can not be resolved with the right join side, the right output is: [c2];
```

## How was this patch tested?

updated tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16100 from cloud-fan/natural.
2016-12-01 11:53:12 -08:00
gatorsmile 2eb093decb [SPARK-17897][SQL] Fixed IsNotNull Constraint Inference Rule
### What changes were proposed in this pull request?
The `constraints` of an operator is the expressions that evaluate to `true` for all the rows produced. That means, the expression result should be neither `false` nor `unknown` (NULL). Thus, we can conclude that `IsNotNull` on all the constraints, which are generated by its own predicates or propagated from the children. The constraint can be a complex expression. For better usage of these constraints, we try to push down `IsNotNull` to the lowest-level expressions (i.e., `Attribute`). `IsNotNull` can be pushed through an expression when it is null intolerant. (When the input is NULL, the null-intolerant expression always evaluates to NULL.)

Below is the existing code we have for `IsNotNull` pushdown.
```Scala
  private def scanNullIntolerantExpr(expr: Expression): Seq[Attribute] = expr match {
    case a: Attribute => Seq(a)
    case _: NullIntolerant | IsNotNull(_: NullIntolerant) =>
      expr.children.flatMap(scanNullIntolerantExpr)
    case _ => Seq.empty[Attribute]
  }
```

**`IsNotNull` itself is not null-intolerant.** It converts `null` to `false`. If the expression does not include any `Not`-like expression, it works; otherwise, it could generate a wrong result. This PR is to fix the above function by removing the `IsNotNull` from the inference. After the fix, when a constraint has a `IsNotNull` expression, we infer new attribute-specific `IsNotNull` constraints if and only if `IsNotNull` appears in the root.

Without the fix, the following test case will return empty.
```Scala
val data = Seq[java.lang.Integer](1, null).toDF("key")
data.filter("not key is not null").show()
```
Before the fix, the optimized plan is like
```
== Optimized Logical Plan ==
Project [value#1 AS key#3]
+- Filter (isnotnull(value#1) && NOT isnotnull(value#1))
   +- LocalRelation [value#1]
```

After the fix, the optimized plan is like
```
== Optimized Logical Plan ==
Project [value#1 AS key#3]
+- Filter NOT isnotnull(value#1)
   +- LocalRelation [value#1]
```

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16067 from gatorsmile/isNotNull2.
2016-11-30 19:40:58 +08:00
Nattavut Sutyanyong 3600635215 [SPARK-18614][SQL] Incorrect predicate pushdown from ExistenceJoin
## What changes were proposed in this pull request?

ExistenceJoin should be treated the same as LeftOuter and LeftAnti, not InnerLike and LeftSemi. This is not currently exposed because the rewrite of [NOT] EXISTS OR ... to ExistenceJoin happens in rule RewritePredicateSubquery, which is in a separate rule set and placed after the rule PushPredicateThroughJoin. During the transformation in the rule PushPredicateThroughJoin, an ExistenceJoin never exists.

The semantics of ExistenceJoin says we need to preserve all the rows from the left table through the join operation as if it is a regular LeftOuter join. The ExistenceJoin augments the LeftOuter operation with a new column called exists, set to true when the join condition in the ON clause is true and false otherwise. The filter of any rows will happen in the Filter operation above the ExistenceJoin.

Example:

A(c1, c2): { (1, 1), (1, 2) }
// B can be any value as it is irrelevant in this example
B(c1): { (NULL) }

select A.*
from   A
where  exists (select 1 from B where A.c1 = A.c2)
       or A.c2=2

In this example, the correct result is all the rows from A. If the pattern ExistenceJoin around line 935 in Optimizer.scala is indeed active, the code will push down the predicate A.c1 = A.c2 to be a Filter on relation A, which will incorrectly filter the row (1,2) from A.

## How was this patch tested?

Since this is not an exposed case, no new test cases is added. The scenario is discovered via a code review of another PR and confirmed to be valid with peer.

Author: Nattavut Sutyanyong <nsy.can@gmail.com>

Closes #16044 from nsyca/spark-18614.
2016-11-29 15:27:43 -08:00
wangzhenhua d57a594b8b [SPARK-18429][SQL] implement a new Aggregate for CountMinSketch
## What changes were proposed in this pull request?

This PR implements a new Aggregate to generate count min sketch, which is a wrapper of CountMinSketch.

## How was this patch tested?

add test cases

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #15877 from wzhfy/cms.
2016-11-29 13:16:46 -08:00
Shuai Lin e64a2047ea [SPARK-16282][SQL] Follow-up: remove "percentile" from temp function detection after implementing it natively
## What changes were proposed in this pull request?

In #15764 we added a mechanism to detect if a function is temporary or not. Hive functions are treated as non-temporary. Of the three hive functions, now "percentile" has been implemented natively, and "hash" has been removed. So we should update the list.

## How was this patch tested?

Unit tests.

Author: Shuai Lin <linshuai2012@gmail.com>

Closes #16049 from lins05/update-temp-function-detect-hive-list.
2016-11-28 20:23:48 -08:00
jiangxingbo 0f5f52a3d1 [SPARK-16282][SQL] Implement percentile SQL function.
## What changes were proposed in this pull request?

Implement percentile SQL function. It computes the exact percentile(s) of expr at pc with range in [0, 1].

## How was this patch tested?

Add a new testsuite `PercentileSuite` to test percentile directly.
Updated related testcases in `ExpressionToSQLSuite`.

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

Closes #14136 from jiangxb1987/percentile.
2016-11-28 11:05:58 -08:00
Herman van Hovell 38e29824d9 [SPARK-18597][SQL] Do not push-down join conditions to the right side of a LEFT ANTI join
## What changes were proposed in this pull request?
We currently push down join conditions of a Left Anti join to both sides of the join. This is similar to Inner, Left Semi and Existence (a specialized left semi) join. The problem is that this changes the semantics of the join; a left anti join filters out rows that matches the join condition.

This PR fixes this by only pushing down conditions to the left hand side of the join. This is similar to the behavior of left outer join.

## How was this patch tested?
Added tests to `FilterPushdownSuite.scala` and created a SQLQueryTestSuite file for left anti joins with a regression test.

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

Closes #16026 from hvanhovell/SPARK-18597.
2016-11-28 07:10:52 -08:00
Herman van Hovell 454b804991 [SPARK-18604][SQL] Make sure CollapseWindow returns the attributes in the same order.
## What changes were proposed in this pull request?
The `CollapseWindow` optimizer rule changes the order of output attributes. This modifies the output of the plan, which the optimizer cannot do. This also breaks things like `collect()` for which we use a `RowEncoder` that assumes that the output attributes of the executed plan are equal to those outputted by the logical plan.

## How was this patch tested?
I have updated an incorrect test in `CollapseWindowSuite`.

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

Closes #16027 from hvanhovell/SPARK-18604.
2016-11-28 02:56:26 -08:00
gatorsmile 07f32c2283 [SPARK-18594][SQL] Name Validation of Databases/Tables
### What changes were proposed in this pull request?
Currently, the name validation checks are limited to table creation. It is enfored by Analyzer rule: `PreWriteCheck`.

However, table renaming and database creation have the same issues. It makes more sense to do the checks in `SessionCatalog`. This PR is to add it into `SessionCatalog`.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16018 from gatorsmile/nameValidate.
2016-11-27 19:43:24 -08:00
Dongjoon Hyun 9c03c56460 [SPARK-17251][SQL] Improve OuterReference to be NamedExpression
## What changes were proposed in this pull request?

Currently, `OuterReference` is not `NamedExpression`. So, it raises 'ClassCastException` when it used in projection lists of IN correlated subqueries. This PR aims to support that by making `OuterReference` as `NamedExpression` to show correct error messages.

```scala
scala> sql("CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES 1, 2 AS t1(a)")
scala> sql("CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES 1 AS t2(b)")
scala> sql("SELECT a FROM t1 WHERE a IN (SELECT a FROM t2)").show
java.lang.ClassCastException: org.apache.spark.sql.catalyst.expressions.OuterReference cannot be cast to org.apache.spark.sql.catalyst.expressions.NamedExpression
```

## How was this patch tested?

Pass the Jenkins test with new test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #16015 from dongjoon-hyun/SPARK-17251-2.
2016-11-26 14:57:48 -08:00
jiangxingbo e2fb9fd365 [SPARK-18436][SQL] isin causing SQL syntax error with JDBC
## What changes were proposed in this pull request?

The expression `in(empty seq)` is invalid in some data source. Since `in(empty seq)` is always false, we should generate `in(empty seq)` to false literal in optimizer.
The sql `SELECT * FROM t WHERE a IN ()` throws a `ParseException` which is consistent with Hive, don't need to change that behavior.

## How was this patch tested?
Add new test case in `OptimizeInSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15977 from jiangxb1987/isin-empty.
2016-11-25 12:44:34 -08:00
Zhenhua Wang 5ecdc7c5c0 [SPARK-18559][SQL] Fix HLL++ with small relative error
## What changes were proposed in this pull request?

In `HyperLogLogPlusPlus`, if the relative error is so small that p >= 19, it will cause ArrayIndexOutOfBoundsException in `THRESHOLDS(p-4)` . We should check `p` and when p >= 19, regress to the original HLL result and use the small range correction they use.

The pr also fixes the upper bound in the log info in `require()`.
The upper bound is computed by:
```
val relativeSD = 1.106d / Math.pow(Math.E, p * Math.log(2.0d) / 2.0d)
```
which is derived from the equation for computing `p`:
```
val p = 2.0d * Math.log(1.106d / relativeSD) / Math.log(2.0d)
```

## How was this patch tested?

add test cases for:
1. checking validity of parameter relatvieSD
2. estimation with smaller relative error so that p >= 19

Author: Zhenhua Wang <wzh_zju@163.com>
Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #15990 from wzhfy/hllppRsd.
2016-11-25 05:02:48 -08:00
Wenchen Fan 84284e8c82 [SPARK-18053][SQL] compare unsafe and safe complex-type values correctly
## What changes were proposed in this pull request?

In Spark SQL, some expression may output safe format values, e.g. `CreateArray`, `CreateStruct`, `Cast`, etc. When we compare 2 values, we should be able to compare safe and unsafe formats.

The `GreaterThan`, `LessThan`, etc. in Spark SQL already handles it, but the `EqualTo` doesn't. This PR fixes it.

## How was this patch tested?

new unit test and regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15929 from cloud-fan/type-aware.
2016-11-23 04:15:19 -08:00
hyukjinkwon 2559fb4b40 [SPARK-18179][SQL] Throws analysis exception with a proper message for unsupported argument types in reflect/java_method function
## What changes were proposed in this pull request?

This PR proposes throwing an `AnalysisException` with a proper message rather than `NoSuchElementException` with the message ` key not found: TimestampType` when unsupported types are given to `reflect` and `java_method` functions.

```scala
spark.range(1).selectExpr("reflect('java.lang.String', 'valueOf', cast('1990-01-01' as timestamp))")
```

produces

**Before**

```
java.util.NoSuchElementException: key not found: TimestampType
  at scala.collection.MapLike$class.default(MapLike.scala:228)
  at scala.collection.AbstractMap.default(Map.scala:59)
  at scala.collection.MapLike$class.apply(MapLike.scala:141)
  at scala.collection.AbstractMap.apply(Map.scala:59)
  at org.apache.spark.sql.catalyst.expressions.CallMethodViaReflection$$anonfun$findMethod$1$$anonfun$apply$1.apply(CallMethodViaReflection.scala:159)
...
```

**After**

```
cannot resolve 'reflect('java.lang.String', 'valueOf', CAST('1990-01-01' AS TIMESTAMP))' due to data type mismatch: arguments from the third require boolean, byte, short, integer, long, float, double or string expressions; line 1 pos 0;
'Project [unresolvedalias(reflect(java.lang.String, valueOf, cast(1990-01-01 as timestamp)), Some(<function1>))]
+- Range (0, 1, step=1, splits=Some(2))
...
```

Added message is,

```
arguments from the third require boolean, byte, short, integer, long, float, double or string expressions
```

## How was this patch tested?

Tests added in `CallMethodViaReflection`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15694 from HyukjinKwon/SPARK-18179.
2016-11-22 22:25:27 -08:00
Wenchen Fan bb152cdfbb [SPARK-18519][SQL] map type can not be used in EqualTo
## What changes were proposed in this pull request?

Technically map type is not orderable, but can be used in equality comparison. However, due to the limitation of the current implementation, map type can't be used in equality comparison so that it can't be join key or grouping key.

This PR makes this limitation explicit, to avoid wrong result.

## How was this patch tested?

updated tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15956 from cloud-fan/map-type.
2016-11-22 09:16:20 -08:00
Herman van Hovell 7ca7a63524 [SPARK-15214][SQL] Code-generation for Generate
## What changes were proposed in this pull request?

This PR adds code generation to `Generate`. It supports two code paths:
- General `TraversableOnce` based iteration. This used for regular `Generator` (code generation supporting) expressions. This code path expects the expression to return a `TraversableOnce[InternalRow]` and it will iterate over the returned collection. This PR adds code generation for the `stack` generator.
- Specialized `ArrayData/MapData` based iteration. This is used for the `explode`, `posexplode` & `inline` functions and operates directly on the `ArrayData`/`MapData` result that the child of the generator returns.

### Benchmarks
I have added some benchmarks and it seems we can create a nice speedup for explode:
#### Environment
```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6
Intel(R) Core(TM) i7-4980HQ CPU  2.80GHz
```
#### Explode Array
##### Before
```
generate explode array:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate explode array wholestage off         7377 / 7607          2.3         439.7       1.0X
generate explode array wholestage on          6055 / 6086          2.8         360.9       1.2X
```
##### After
```
generate explode array:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate explode array wholestage off         7432 / 7696          2.3         443.0       1.0X
generate explode array wholestage on           631 /  646         26.6          37.6      11.8X
```
#### Explode Map
##### Before
```
generate explode map:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate explode map wholestage off         12792 / 12848          1.3         762.5       1.0X
generate explode map wholestage on          11181 / 11237          1.5         666.5       1.1X
```
##### After
```
generate explode map:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate explode map wholestage off         10949 / 10972          1.5         652.6       1.0X
generate explode map wholestage on             870 /  913         19.3          51.9      12.6X
```
#### Posexplode
##### Before
```
generate posexplode array:               Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate posexplode array wholestage off      7547 / 7580          2.2         449.8       1.0X
generate posexplode array wholestage on       5786 / 5838          2.9         344.9       1.3X
```
##### After
```
generate posexplode array:               Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate posexplode array wholestage off      7535 / 7548          2.2         449.1       1.0X
generate posexplode array wholestage on        620 /  624         27.1          37.0      12.1X
```
#### Inline
##### Before
```
generate inline array:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate inline array wholestage off          6935 / 6978          2.4         413.3       1.0X
generate inline array wholestage on           6360 / 6400          2.6         379.1       1.1X
```
##### After
```
generate inline array:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate inline array wholestage off          6940 / 6966          2.4         413.6       1.0X
generate inline array wholestage on           1002 / 1012         16.7          59.7       6.9X
```
#### Stack
##### Before
```
generate stack:                          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate stack wholestage off               12980 / 13104          1.3         773.7       1.0X
generate stack wholestage on                11566 / 11580          1.5         689.4       1.1X
```
##### After
```
generate stack:                          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate stack wholestage off               12875 / 12949          1.3         767.4       1.0X
generate stack wholestage on                   840 /  845         20.0          50.0      15.3X
```
## How was this patch tested?

Existing tests.

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

Closes #13065 from hvanhovell/SPARK-15214.
2016-11-19 23:55:09 -08:00
Xianyang Liu 7569cf6cb8
[SPARK-18420][BUILD] Fix the errors caused by lint check in Java
## What changes were proposed in this pull request?

Small fix, fix the errors caused by lint check in Java

- Clear unused objects and `UnusedImports`.
- Add comments around the method `finalize` of `NioBufferedFileInputStream`to turn off checkstyle.
- Cut the line which is longer than 100 characters into two lines.

## How was this patch tested?
Travis CI.
```
$ build/mvn -T 4 -q -DskipTests -Pyarn -Phadoop-2.3 -Pkinesis-asl -Phive -Phive-thriftserver install
$ dev/lint-java
```
Before:
```
Checkstyle checks failed at following occurrences:
[ERROR] src/main/java/org/apache/spark/network/util/TransportConf.java:[21,8] (imports) UnusedImports: Unused import - org.apache.commons.crypto.cipher.CryptoCipherFactory.
[ERROR] src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java:[516,5] (modifier) RedundantModifier: Redundant 'public' modifier.
[ERROR] src/main/java/org/apache/spark/io/NioBufferedFileInputStream.java:[133] (coding) NoFinalizer: Avoid using finalizer method.
[ERROR] src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeMapData.java:[71] (sizes) LineLength: Line is longer than 100 characters (found 113).
[ERROR] src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java:[112] (sizes) LineLength: Line is longer than 100 characters (found 110).
[ERROR] src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java:[31,17] (modifier) ModifierOrder: 'static' modifier out of order with the JLS suggestions.
[ERROR]src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionWithElasticNetExample.java:[64] (sizes) LineLength: Line is longer than 100 characters (found 103).
[ERROR] src/main/java/org/apache/spark/examples/ml/JavaInteractionExample.java:[22,8] (imports) UnusedImports: Unused import - org.apache.spark.ml.linalg.Vectors.
[ERROR] src/main/java/org/apache/spark/examples/ml/JavaInteractionExample.java:[51] (regexp) RegexpSingleline: No trailing whitespace allowed.
```

After:
```
$ build/mvn -T 4 -q -DskipTests -Pyarn -Phadoop-2.3 -Pkinesis-asl -Phive -Phive-thriftserver install
$ dev/lint-java
Using `mvn` from path: /home/travis/build/ConeyLiu/spark/build/apache-maven-3.3.9/bin/mvn
Checkstyle checks passed.
```

Author: Xianyang Liu <xyliu0530@icloud.com>

Closes #15865 from ConeyLiu/master.
2016-11-16 11:59:00 +00:00
Herman van Hovell f14ae4900a [SPARK-18300][SQL] Do not apply foldable propagation with expand as a child.
## What changes were proposed in this pull request?
The `FoldablePropagation` optimizer rule, pulls foldable values out from under an `Expand`. This breaks the `Expand` in two ways:

- It rewrites the output attributes of the `Expand`. We explicitly define output attributes for `Expand`, these are (unfortunately) considered as part of the expressions of the `Expand` and can be rewritten.
- Expand can actually change the column (it will typically re-use the attributes or the underlying plan). This means that we cannot safely propagate the expressions from under an `Expand`.

This PR fixes this and (hopefully) other issues by explicitly whitelisting allowed operators.

## How was this patch tested?
Added tests to `FoldablePropagationSuite` and to `SQLQueryTestSuite`.

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

Closes #15857 from hvanhovell/SPARK-18300.
2016-11-15 06:59:25 -08:00
Ryan Blue 6e95325fc3 [SPARK-18387][SQL] Add serialization to checkEvaluation.
## What changes were proposed in this pull request?

This removes the serialization test from RegexpExpressionsSuite and
replaces it by serializing all expressions in checkEvaluation.

This also fixes math constant expressions by making LeafMathExpression
Serializable and fixes NumberFormat values that are null or invalid
after serialization.

## How was this patch tested?

This patch is to tests.

Author: Ryan Blue <blue@apache.org>

Closes #15847 from rdblue/SPARK-18387-fix-serializable-expressions.
2016-11-11 13:52:10 -08:00
Eric Liang a3356343cb [SPARK-18185] Fix all forms of INSERT / OVERWRITE TABLE for Datasource tables
## What changes were proposed in this pull request?

As of current 2.1, INSERT OVERWRITE with dynamic partitions against a Datasource table will overwrite the entire table instead of only the partitions matching the static keys, as in Hive. It also doesn't respect custom partition locations.

This PR adds support for all these operations to Datasource tables managed by the Hive metastore. It is implemented as follows
- During planning time, the full set of partitions affected by an INSERT or OVERWRITE command is read from the Hive metastore.
- The planner identifies any partitions with custom locations and includes this in the write task metadata.
- FileFormatWriter tasks refer to this custom locations map when determining where to write for dynamic partition output.
- When the write job finishes, the set of written partitions is compared against the initial set of matched partitions, and the Hive metastore is updated to reflect the newly added / removed partitions.

It was necessary to introduce a method for staging files with absolute output paths to `FileCommitProtocol`. These files are not handled by the Hadoop output committer but are moved to their final locations when the job commits.

The overwrite behavior of legacy Datasource tables is also changed: no longer will the entire table be overwritten if a partial partition spec is present.

cc cloud-fan yhuai

## How was this patch tested?

Unit tests, existing tests.

Author: Eric Liang <ekl@databricks.com>
Author: Wenchen Fan <wenchen@databricks.com>

Closes #15814 from ericl/sc-5027.
2016-11-10 17:00:43 -08:00
Wenchen Fan 2f7461f313 [SPARK-17990][SPARK-18302][SQL] correct several partition related behaviours of ExternalCatalog
## What changes were proposed in this pull request?

This PR corrects several partition related behaviors of `ExternalCatalog`:

1. default partition location should not always lower case the partition column names in path string(fix `HiveExternalCatalog`)
2. rename partition should not always lower case the partition column names in updated partition path string(fix `HiveExternalCatalog`)
3. rename partition should update the partition location only for managed table(fix `InMemoryCatalog`)
4. create partition with existing directory should be fine(fix `InMemoryCatalog`)
5. create partition with non-existing directory should create that directory(fix `InMemoryCatalog`)
6. drop partition from external table should not delete the directory(fix `InMemoryCatalog`)

## How was this patch tested?

new tests in `ExternalCatalogSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15797 from cloud-fan/partition.
2016-11-10 13:42:48 -08:00
Ryan Blue d4028de976 [SPARK-18368][SQL] Fix regexp replace when serialized
## What changes were proposed in this pull request?

This makes the result value both transient and lazy, so that if the RegExpReplace object is initialized then serialized, `result: StringBuffer` will be correctly initialized.

## How was this patch tested?

* Verified that this patch fixed the query that found the bug.
* Added a test case that fails without the fix.

Author: Ryan Blue <blue@apache.org>

Closes #15834 from rdblue/SPARK-18368-fix-regexp-replace.
2016-11-09 11:00:53 -08:00
Yin Huai 47636618a5 Revert "[SPARK-18368] Fix regexp_replace with task serialization."
This reverts commit b9192bb3ff.
2016-11-09 10:47:29 -08:00
Ryan Blue b9192bb3ff [SPARK-18368] Fix regexp_replace with task serialization.
## What changes were proposed in this pull request?

This makes the result value both transient and lazy, so that if the RegExpReplace object is initialized then serialized, `result: StringBuffer` will be correctly initialized.

## How was this patch tested?

* Verified that this patch fixed the query that found the bug.
* Added a test case that fails without the fix.

Author: Ryan Blue <blue@apache.org>

Closes #15816 from rdblue/SPARK-18368-fix-regexp-replace.
2016-11-08 23:47:48 -08:00
jiangxingbo 344dcad701 [SPARK-17868][SQL] Do not use bitmasks during parsing and analysis of CUBE/ROLLUP/GROUPING SETS
## What changes were proposed in this pull request?

We generate bitmasks for grouping sets during the parsing process, and use these during analysis. These bitmasks are difficult to work with in practice and have lead to numerous bugs. This PR removes these and use actual sets instead, however we still need to generate these offsets for the grouping_id.

This PR does the following works:
1. Replace bitmasks by actual grouping sets durning Parsing/Analysis stage of CUBE/ROLLUP/GROUPING SETS;
2. Add new testsuite `ResolveGroupingAnalyticsSuite` to test the `Analyzer.ResolveGroupingAnalytics` rule directly;
3. Fix a minor bug in `ResolveGroupingAnalytics`.
## How was this patch tested?

By existing test cases, and add new testsuite `ResolveGroupingAnalyticsSuite` to test directly.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15484 from jiangxb1987/group-set.
2016-11-08 15:11:03 +01:00
Kazuaki Ishizaki 47731e1865 [SPARK-18207][SQL] Fix a compilation error due to HashExpression.doGenCode
## What changes were proposed in this pull request?

This PR avoids a compilation error due to more than 64KB Java byte code size. This error occur since  generate java code for computing a hash value for a row is too big. This PR fixes this compilation error by splitting a big code chunk into multiple methods by calling `CodegenContext.splitExpression` at `HashExpression.doGenCode`

The test case requires a calculation of hash code for a row that includes 1000 String fields. `HashExpression.doGenCode` generate a lot of Java code for this computation into one function. As a result, the size of the corresponding Java bytecode is more than 64 KB.

Generated code without this PR
````java
/* 027 */   public UnsafeRow apply(InternalRow i) {
/* 028 */     boolean isNull = false;
/* 029 */
/* 030 */     int value1 = 42;
/* 031 */
/* 032 */     boolean isNull2 = i.isNullAt(0);
/* 033 */     UTF8String value2 = isNull2 ? null : (i.getUTF8String(0));
/* 034 */     if (!isNull2) {
/* 035 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value2.getBaseObject(), value2.getBaseOffset(), value2.numBytes(), value1);
/* 036 */     }
/* 037 */
/* 038 */
/* 039 */     boolean isNull3 = i.isNullAt(1);
/* 040 */     UTF8String value3 = isNull3 ? null : (i.getUTF8String(1));
/* 041 */     if (!isNull3) {
/* 042 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value3.getBaseObject(), value3.getBaseOffset(), value3.numBytes(), value1);
/* 043 */     }
/* 044 */
/* 045 */
...
/* 7024 */
/* 7025 */     boolean isNull1001 = i.isNullAt(999);
/* 7026 */     UTF8String value1001 = isNull1001 ? null : (i.getUTF8String(999));
/* 7027 */     if (!isNull1001) {
/* 7028 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value1001.getBaseObject(), value1001.getBaseOffset(), value1001.numBytes(), value1);
/* 7029 */     }
/* 7030 */
/* 7031 */
/* 7032 */     boolean isNull1002 = i.isNullAt(1000);
/* 7033 */     UTF8String value1002 = isNull1002 ? null : (i.getUTF8String(1000));
/* 7034 */     if (!isNull1002) {
/* 7035 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value1002.getBaseObject(), value1002.getBaseOffset(), value1002.numBytes(), value1);
/* 7036 */     }
````

Generated code with this PR
````java
/* 3807 */   private void apply_249(InternalRow i) {
/* 3808 */
/* 3809 */     boolean isNull998 = i.isNullAt(996);
/* 3810 */     UTF8String value998 = isNull998 ? null : (i.getUTF8String(996));
/* 3811 */     if (!isNull998) {
/* 3812 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value998.getBaseObject(), value998.getBaseOffset(), value998.numBytes(), value1);
/* 3813 */     }
/* 3814 */
/* 3815 */     boolean isNull999 = i.isNullAt(997);
/* 3816 */     UTF8String value999 = isNull999 ? null : (i.getUTF8String(997));
/* 3817 */     if (!isNull999) {
/* 3818 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value999.getBaseObject(), value999.getBaseOffset(), value999.numBytes(), value1);
/* 3819 */     }
/* 3820 */
/* 3821 */     boolean isNull1000 = i.isNullAt(998);
/* 3822 */     UTF8String value1000 = isNull1000 ? null : (i.getUTF8String(998));
/* 3823 */     if (!isNull1000) {
/* 3824 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value1000.getBaseObject(), value1000.getBaseOffset(), value1000.numBytes(), value1);
/* 3825 */     }
/* 3826 */
/* 3827 */     boolean isNull1001 = i.isNullAt(999);
/* 3828 */     UTF8String value1001 = isNull1001 ? null : (i.getUTF8String(999));
/* 3829 */     if (!isNull1001) {
/* 3830 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value1001.getBaseObject(), value1001.getBaseOffset(), value1001.numBytes(), value1);
/* 3831 */     }
/* 3832 */
/* 3833 */   }
/* 3834 */
...
/* 4532 */   private void apply_0(InternalRow i) {
/* 4533 */
/* 4534 */     boolean isNull2 = i.isNullAt(0);
/* 4535 */     UTF8String value2 = isNull2 ? null : (i.getUTF8String(0));
/* 4536 */     if (!isNull2) {
/* 4537 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value2.getBaseObject(), value2.getBaseOffset(), value2.numBytes(), value1);
/* 4538 */     }
/* 4539 */
/* 4540 */     boolean isNull3 = i.isNullAt(1);
/* 4541 */     UTF8String value3 = isNull3 ? null : (i.getUTF8String(1));
/* 4542 */     if (!isNull3) {
/* 4543 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value3.getBaseObject(), value3.getBaseOffset(), value3.numBytes(), value1);
/* 4544 */     }
/* 4545 */
/* 4546 */     boolean isNull4 = i.isNullAt(2);
/* 4547 */     UTF8String value4 = isNull4 ? null : (i.getUTF8String(2));
/* 4548 */     if (!isNull4) {
/* 4549 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value4.getBaseObject(), value4.getBaseOffset(), value4.numBytes(), value1);
/* 4550 */     }
/* 4551 */
/* 4552 */     boolean isNull5 = i.isNullAt(3);
/* 4553 */     UTF8String value5 = isNull5 ? null : (i.getUTF8String(3));
/* 4554 */     if (!isNull5) {
/* 4555 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value5.getBaseObject(), value5.getBaseOffset(), value5.numBytes(), value1);
/* 4556 */     }
/* 4557 */
/* 4558 */   }
...
/* 7344 */   public UnsafeRow apply(InternalRow i) {
/* 7345 */     boolean isNull = false;
/* 7346 */
/* 7347 */     value1 = 42;
/* 7348 */     apply_0(i);
/* 7349 */     apply_1(i);
...
/* 7596 */     apply_248(i);
/* 7597 */     apply_249(i);
/* 7598 */     apply_250(i);
/* 7599 */     apply_251(i);
...
````

## How was this patch tested?

Add a new test in `DataFrameSuite`

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

Closes #15745 from kiszk/SPARK-18207.
2016-11-08 12:01:54 +01:00
gatorsmile 1da64e1fa0 [SPARK-18217][SQL] Disallow creating permanent views based on temporary views or UDFs
### What changes were proposed in this pull request?
Based on the discussion in [SPARK-18209](https://issues.apache.org/jira/browse/SPARK-18209). It doesn't really make sense to create permanent views based on temporary views or temporary UDFs.

To disallow the supports and issue the exceptions, this PR needs to detect whether a temporary view/UDF is being used when defining a permanent view. Basically, this PR can be split to two sub-tasks:

**Task 1:** detecting a temporary view from the query plan of view definition.
When finding an unresolved temporary view, Analyzer replaces it by a `SubqueryAlias` with the corresponding logical plan, which is stored in an in-memory HashMap. After replacement, it is impossible to detect whether the `SubqueryAlias` is added/generated from a temporary view. Thus, to detect the usage of a temporary view in view definition, this PR traverses the unresolved logical plan and uses the name of an `UnresolvedRelation` to detect whether it is a (global) temporary view.

**Task 2:** detecting a temporary UDF from the query plan of view definition.
Detecting usage of a temporary UDF in view definition is not straightfoward.

First, in the analyzed plan, we are having different forms to represent the functions. More importantly, some classes (e.g., `HiveGenericUDF`) are not accessible from `CreateViewCommand`, which is part of  `sql/core`. Thus, we used the unanalyzed plan `child` of `CreateViewCommand` to detect the usage of a temporary UDF. Because the plan has already been successfully analyzed, we can assume the functions have been defined/registered.

Second, in Spark, the functions have four forms: Spark built-in functions, built-in hash functions, permanent UDFs and temporary UDFs. We do not have any direct way to determine whether a function is temporary or not. Thus, we introduced a function `isTemporaryFunction` in `SessionCatalog`. This function contains the detailed logics to determine whether a function is temporary or not.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15764 from gatorsmile/blockTempFromPermViewCreation.
2016-11-07 18:34:21 -08:00
hyukjinkwon 3eda05703f [SPARK-18295][SQL] Make to_json function null safe (matching it to from_json)
## What changes were proposed in this pull request?

This PR proposes to match up the behaviour of `to_json` to `from_json` function for null-safety.

Currently, it throws `NullPointException` but this PR fixes this to produce `null` instead.

with the data below:

```scala
import spark.implicits._

val df = Seq(Some(Tuple1(Tuple1(1))), None).toDF("a")
df.show()
```

```
+----+
|   a|
+----+
| [1]|
|null|
+----+
```

the codes below

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

df.select(to_json($"a")).show()
```

produces..

**Before**

throws `NullPointException` as below:

```
java.lang.NullPointerException
  at org.apache.spark.sql.catalyst.json.JacksonGenerator.org$apache$spark$sql$catalyst$json$JacksonGenerator$$writeFields(JacksonGenerator.scala:138)
  at org.apache.spark.sql.catalyst.json.JacksonGenerator$$anonfun$write$1.apply$mcV$sp(JacksonGenerator.scala:194)
  at org.apache.spark.sql.catalyst.json.JacksonGenerator.org$apache$spark$sql$catalyst$json$JacksonGenerator$$writeObject(JacksonGenerator.scala:131)
  at org.apache.spark.sql.catalyst.json.JacksonGenerator.write(JacksonGenerator.scala:193)
  at org.apache.spark.sql.catalyst.expressions.StructToJson.eval(jsonExpressions.scala:544)
  at org.apache.spark.sql.catalyst.expressions.Alias.eval(namedExpressions.scala:142)
  at org.apache.spark.sql.catalyst.expressions.InterpretedProjection.apply(Projection.scala:48)
  at org.apache.spark.sql.catalyst.expressions.InterpretedProjection.apply(Projection.scala:30)
  at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
```

**After**

```
+---------------+
|structtojson(a)|
+---------------+
|       {"_1":1}|
|           null|
+---------------+
```

## How was this patch tested?

Unit test in `JsonExpressionsSuite.scala` and `JsonFunctionsSuite.scala`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15792 from HyukjinKwon/SPARK-18295.
2016-11-07 16:54:40 -08:00
Kazuaki Ishizaki 19cf208063 [SPARK-17490][SQL] Optimize SerializeFromObject() for a primitive array
## What changes were proposed in this pull request?

Waiting for merging #13680

This PR optimizes `SerializeFromObject()` for an primitive array. This is derived from #13758 to address one of problems by using a simple way in #13758.

The current implementation always generates `GenericArrayData` from `SerializeFromObject()` for any type of an array in a logical plan. This involves a boxing at a constructor of `GenericArrayData` when `SerializedFromObject()` has an primitive array.

This PR enables to generate `UnsafeArrayData` from `SerializeFromObject()` for a primitive array. It can avoid boxing to create an instance of `ArrayData` in the generated code by Catalyst.

This PR also generate `UnsafeArrayData` in a case for `RowEncoder.serializeFor` or `CatalystTypeConverters.createToCatalystConverter`.

Performance improvement of `SerializeFromObject()` is up to 2.0x

```
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)

Without this PR
Write an array in Dataset:               Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            556 /  608         15.1          66.3       1.0X
Double                                        1668 / 1746          5.0         198.8       0.3X

with this PR
Write an array in Dataset:               Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            352 /  401         23.8          42.0       1.0X
Double                                         821 /  885         10.2          97.9       0.4X
```

Here is an example program that will happen in mllib as described in [SPARK-16070](https://issues.apache.org/jira/browse/SPARK-16070).

```
sparkContext.parallelize(Seq(Array(1, 2)), 1).toDS.map(e => e).show
```

Generated code before applying this PR

``` java
/* 039 */   protected void processNext() throws java.io.IOException {
/* 040 */     while (inputadapter_input.hasNext()) {
/* 041 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 042 */       int[] inputadapter_value = (int[])inputadapter_row.get(0, null);
/* 043 */
/* 044 */       Object mapelements_obj = ((Expression) references[0]).eval(null);
/* 045 */       scala.Function1 mapelements_value1 = (scala.Function1) mapelements_obj;
/* 046 */
/* 047 */       boolean mapelements_isNull = false || false;
/* 048 */       int[] mapelements_value = null;
/* 049 */       if (!mapelements_isNull) {
/* 050 */         Object mapelements_funcResult = null;
/* 051 */         mapelements_funcResult = mapelements_value1.apply(inputadapter_value);
/* 052 */         if (mapelements_funcResult == null) {
/* 053 */           mapelements_isNull = true;
/* 054 */         } else {
/* 055 */           mapelements_value = (int[]) mapelements_funcResult;
/* 056 */         }
/* 057 */
/* 058 */       }
/* 059 */       mapelements_isNull = mapelements_value == null;
/* 060 */
/* 061 */       serializefromobject_argIsNulls[0] = mapelements_isNull;
/* 062 */       serializefromobject_argValue = mapelements_value;
/* 063 */
/* 064 */       boolean serializefromobject_isNull = false;
/* 065 */       for (int idx = 0; idx < 1; idx++) {
/* 066 */         if (serializefromobject_argIsNulls[idx]) { serializefromobject_isNull = true; break; }
/* 067 */       }
/* 068 */
/* 069 */       final ArrayData serializefromobject_value = serializefromobject_isNull ? null : new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_argValue);
/* 070 */       serializefromobject_holder.reset();
/* 071 */
/* 072 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 073 */
/* 074 */       if (serializefromobject_isNull) {
/* 075 */         serializefromobject_rowWriter.setNullAt(0);
/* 076 */       } else {
/* 077 */         // Remember the current cursor so that we can calculate how many bytes are
/* 078 */         // written later.
/* 079 */         final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
/* 080 */
/* 081 */         if (serializefromobject_value instanceof UnsafeArrayData) {
/* 082 */           final int serializefromobject_sizeInBytes = ((UnsafeArrayData) serializefromobject_value).getSizeInBytes();
/* 083 */           // grow the global buffer before writing data.
/* 084 */           serializefromobject_holder.grow(serializefromobject_sizeInBytes);
/* 085 */           ((UnsafeArrayData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 086 */           serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
/* 087 */
/* 088 */         } else {
/* 089 */           final int serializefromobject_numElements = serializefromobject_value.numElements();
/* 090 */           serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 4);
/* 091 */
/* 092 */           for (int serializefromobject_index = 0; serializefromobject_index < serializefromobject_numElements; serializefromobject_index++) {
/* 093 */             if (serializefromobject_value.isNullAt(serializefromobject_index)) {
/* 094 */               serializefromobject_arrayWriter.setNullInt(serializefromobject_index);
/* 095 */             } else {
/* 096 */               final int serializefromobject_element = serializefromobject_value.getInt(serializefromobject_index);
/* 097 */               serializefromobject_arrayWriter.write(serializefromobject_index, serializefromobject_element);
/* 098 */             }
/* 099 */           }
/* 100 */         }
/* 101 */
/* 102 */         serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
/* 103 */       }
/* 104 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
/* 105 */       append(serializefromobject_result);
/* 106 */       if (shouldStop()) return;
/* 107 */     }
/* 108 */   }
/* 109 */ }
```

Generated code after applying this PR

``` java
/* 035 */   protected void processNext() throws java.io.IOException {
/* 036 */     while (inputadapter_input.hasNext()) {
/* 037 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 038 */       int[] inputadapter_value = (int[])inputadapter_row.get(0, null);
/* 039 */
/* 040 */       Object mapelements_obj = ((Expression) references[0]).eval(null);
/* 041 */       scala.Function1 mapelements_value1 = (scala.Function1) mapelements_obj;
/* 042 */
/* 043 */       boolean mapelements_isNull = false || false;
/* 044 */       int[] mapelements_value = null;
/* 045 */       if (!mapelements_isNull) {
/* 046 */         Object mapelements_funcResult = null;
/* 047 */         mapelements_funcResult = mapelements_value1.apply(inputadapter_value);
/* 048 */         if (mapelements_funcResult == null) {
/* 049 */           mapelements_isNull = true;
/* 050 */         } else {
/* 051 */           mapelements_value = (int[]) mapelements_funcResult;
/* 052 */         }
/* 053 */
/* 054 */       }
/* 055 */       mapelements_isNull = mapelements_value == null;
/* 056 */
/* 057 */       boolean serializefromobject_isNull = mapelements_isNull;
/* 058 */       final ArrayData serializefromobject_value = serializefromobject_isNull ? null : org.apache.spark.sql.catalyst.expressions.UnsafeArrayData.fromPrimitiveArray(mapelements_value);
/* 059 */       serializefromobject_isNull = serializefromobject_value == null;
/* 060 */       serializefromobject_holder.reset();
/* 061 */
/* 062 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 063 */
/* 064 */       if (serializefromobject_isNull) {
/* 065 */         serializefromobject_rowWriter.setNullAt(0);
/* 066 */       } else {
/* 067 */         // Remember the current cursor so that we can calculate how many bytes are
/* 068 */         // written later.
/* 069 */         final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
/* 070 */
/* 071 */         if (serializefromobject_value instanceof UnsafeArrayData) {
/* 072 */           final int serializefromobject_sizeInBytes = ((UnsafeArrayData) serializefromobject_value).getSizeInBytes();
/* 073 */           // grow the global buffer before writing data.
/* 074 */           serializefromobject_holder.grow(serializefromobject_sizeInBytes);
/* 075 */           ((UnsafeArrayData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 076 */           serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
/* 077 */
/* 078 */         } else {
/* 079 */           final int serializefromobject_numElements = serializefromobject_value.numElements();
/* 080 */           serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 4);
/* 081 */
/* 082 */           for (int serializefromobject_index = 0; serializefromobject_index < serializefromobject_numElements; serializefromobject_index++) {
/* 083 */             if (serializefromobject_value.isNullAt(serializefromobject_index)) {
/* 084 */               serializefromobject_arrayWriter.setNullInt(serializefromobject_index);
/* 085 */             } else {
/* 086 */               final int serializefromobject_element = serializefromobject_value.getInt(serializefromobject_index);
/* 087 */               serializefromobject_arrayWriter.write(serializefromobject_index, serializefromobject_element);
/* 088 */             }
/* 089 */           }
/* 090 */         }
/* 091 */
/* 092 */         serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
/* 093 */       }
/* 094 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
/* 095 */       append(serializefromobject_result);
/* 096 */       if (shouldStop()) return;
/* 097 */     }
/* 098 */   }
/* 099 */ }
```
## How was this patch tested?

Added a test in `DatasetSuite`, `RowEncoderSuite`, and `CatalystTypeConvertersSuite`

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

Closes #15044 from kiszk/SPARK-17490.
2016-11-08 00:14:57 +01:00
Reynold Xin 9db06c442c [SPARK-18296][SQL] Use consistent naming for expression test suites
## What changes were proposed in this pull request?
We have an undocumented naming convention to call expression unit tests ExpressionsSuite, and the end-to-end tests FunctionsSuite. It'd be great to make all test suites consistent with this naming convention.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #15793 from rxin/SPARK-18296.
2016-11-06 22:44:55 -08:00
Wenchen Fan 46b2e49993 [SPARK-18173][SQL] data source tables should support truncating partition
## What changes were proposed in this pull request?

Previously `TRUNCATE TABLE ... PARTITION` will always truncate the whole table for data source tables, this PR fixes it and improve `InMemoryCatalog` to make this command work with it.
## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15688 from cloud-fan/truncate.
2016-11-06 18:57:13 -08:00
hyukjinkwon 340f09d100
[SPARK-17854][SQL] rand/randn allows null/long as input seed
## What changes were proposed in this pull request?

This PR proposes `rand`/`randn` accept `null` as input in Scala/SQL and `LongType` as input in SQL. In this case, it treats the values as `0`.

So, this PR includes both changes below:
- `null` support

  It seems MySQL also accepts this.

  ``` sql
  mysql> select rand(0);
  +---------------------+
  | rand(0)             |
  +---------------------+
  | 0.15522042769493574 |
  +---------------------+
  1 row in set (0.00 sec)

  mysql> select rand(NULL);
  +---------------------+
  | rand(NULL)          |
  +---------------------+
  | 0.15522042769493574 |
  +---------------------+
  1 row in set (0.00 sec)
  ```

  and also Hive does according to [HIVE-14694](https://issues.apache.org/jira/browse/HIVE-14694)

  So the codes below:

  ``` scala
  spark.range(1).selectExpr("rand(null)").show()
  ```

  prints..

  **Before**

  ```
    Input argument to rand must be an integer literal.;; line 1 pos 0
  org.apache.spark.sql.AnalysisException: Input argument to rand must be an integer literal.;; line 1 pos 0
  at org.apache.spark.sql.catalyst.analysis.FunctionRegistry$$anonfun$5.apply(FunctionRegistry.scala:465)
  at org.apache.spark.sql.catalyst.analysis.FunctionRegistry$$anonfun$5.apply(FunctionRegistry.scala:444)
  ```

  **After**

  ```
    +-----------------------+
    |rand(CAST(NULL AS INT))|
    +-----------------------+
    |    0.13385709732307427|
    +-----------------------+
  ```
- `LongType` support in SQL.

  In addition, it make the function allows to take `LongType` consistently within Scala/SQL.

  In more details, the codes below:

  ``` scala
  spark.range(1).select(rand(1), rand(1L)).show()
  spark.range(1).selectExpr("rand(1)", "rand(1L)").show()
  ```

  prints..

  **Before**

  ```
  +------------------+------------------+
  |           rand(1)|           rand(1)|
  +------------------+------------------+
  |0.2630967864682161|0.2630967864682161|
  +------------------+------------------+

  Input argument to rand must be an integer literal.;; line 1 pos 0
  org.apache.spark.sql.AnalysisException: Input argument to rand must be an integer literal.;; line 1 pos 0
  at org.apache.spark.sql.catalyst.analysis.FunctionRegistry$$anonfun$5.apply(FunctionRegistry.scala:465)
  at
  ```

  **After**

  ```
  +------------------+------------------+
  |           rand(1)|           rand(1)|
  +------------------+------------------+
  |0.2630967864682161|0.2630967864682161|
  +------------------+------------------+

  +------------------+------------------+
  |           rand(1)|           rand(1)|
  +------------------+------------------+
  |0.2630967864682161|0.2630967864682161|
  +------------------+------------------+
  ```
## How was this patch tested?

Unit tests in `DataFrameSuite.scala` and `RandomSuite.scala`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15432 from HyukjinKwon/SPARK-17854.
2016-11-06 14:11:37 +00:00
Reynold Xin e2648d3557 [SPARK-18287][SQL] Move hash expressions from misc.scala into hash.scala
## What changes were proposed in this pull request?
As the title suggests, this patch moves hash expressions from misc.scala into hash.scala, to make it easier to find the hash functions. I wanted to do this a while ago but decided to wait for the branch-2.1 cut so the chance of conflicts will be smaller.

## How was this patch tested?
Test cases were also moved out of MiscFunctionsSuite into HashExpressionsSuite.

Author: Reynold Xin <rxin@databricks.com>

Closes #15784 from rxin/SPARK-18287.
2016-11-05 11:29:17 +01:00
Wenchen Fan 95ec4e25bb [SPARK-17183][SPARK-17983][SPARK-18101][SQL] put hive serde table schema to table properties like data source table
## What changes were proposed in this pull request?

For data source tables, we will put its table schema, partition columns, etc. to table properties, to work around some hive metastore issues, e.g. not case-preserving, bad decimal type support, etc.

We should also do this for hive serde tables, to reduce the difference between hive serde tables and data source tables, e.g. column names should be case preserving.
## How was this patch tested?

existing tests, and a new test in `HiveExternalCatalog`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14750 from cloud-fan/minor1.
2016-11-05 00:58:50 -07:00
Burak Yavuz 6e27018157 [SPARK-18260] Make from_json null safe
## What changes were proposed in this pull request?

`from_json` is currently not safe against `null` rows. This PR adds a fix and a regression test for it.

## How was this patch tested?

Regression test

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #15771 from brkyvz/json_fix.
2016-11-05 00:07:51 -07:00
Reynold Xin b17057c0a6 [SPARK-18244][SQL] Rename partitionProviderIsHive -> tracksPartitionsInCatalog
## What changes were proposed in this pull request?
This patch renames partitionProviderIsHive to tracksPartitionsInCatalog, as the old name was too Hive specific.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #15750 from rxin/SPARK-18244.
2016-11-03 11:48:05 -07:00
Daoyuan Wang 96cc1b5675 [SPARK-17122][SQL] support drop current database
## What changes were proposed in this pull request?

In Spark 1.6 and earlier, we can drop the database we are using. In Spark 2.0, native implementation prevent us from dropping current database, which may break some old queries. This PR would re-enable the feature.
## How was this patch tested?

one new unit test in `SessionCatalogSuite`.

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

Closes #15011 from adrian-wang/dropcurrent.
2016-11-03 00:18:03 -07:00
gatorsmile 9ddec8636c [SPARK-18175][SQL] Improve the test case coverage of implicit type casting
### What changes were proposed in this pull request?

So far, we have limited test case coverage about implicit type casting. We need to draw a matrix to find all the possible casting pairs.
- Reorged the existing test cases
- Added all the possible type casting pairs
- Drawed a matrix to show the implicit type casting. The table is very wide. Maybe hard to review. Thus, you also can access the same table via the link to [a google sheet](https://docs.google.com/spreadsheets/d/19PS4ikrs-Yye_mfu-rmIKYGnNe-NmOTt5DDT1fOD3pI/edit?usp=sharing).

SourceType\CastToType | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | BinaryType | BooleanType | StringType | DateType | TimestampType | ArrayType | MapType | StructType | NullType | CalendarIntervalType | DecimalType | NumericType | IntegralType
------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ |  -----------
**ByteType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X    | X    | StringType | X    | X    | X    | X    | X    | X    | X    | DecimalType(3, 0) | ByteType | ByteType
**ShortType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X    | X    | StringType | X    | X    | X    | X    | X    | X    | X    | DecimalType(5, 0) | ShortType | ShortType
**IntegerType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X    | X    | StringType | X    | X    | X    | X    | X    | X    | X    | DecimalType(10, 0) | IntegerType | IntegerType
**LongType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X    | X    | StringType | X    | X    | X    | X    | X    | X    | X    | DecimalType(20, 0) | LongType | LongType
**DoubleType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X    | X    | StringType | X    | X    | X    | X    | X    | X    | X    | DecimalType(30, 15) | DoubleType | IntegerType
**FloatType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X    | X    | StringType | X    | X    | X    | X    | X    | X    | X    | DecimalType(14, 7) | FloatType | IntegerType
**Dec(10, 2)** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X    | X    | StringType | X    | X    | X    | X    | X    | X    | X    | DecimalType(10, 2) | Dec(10, 2) | IntegerType
**BinaryType** | X    | X    | X    | X    | X    | X    | X    | BinaryType | X    | StringType | X    | X    | X    | X    | X    | X    | X    | X    | X    | X
**BooleanType** | X    | X    | X    | X    | X    | X    | X    | X    | BooleanType | StringType | X    | X    | X    | X    | X    | X    | X    | X    | X    | X
**StringType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | BinaryType | X    | StringType | DateType | TimestampType | X    | X    | X    | X    | X    | DecimalType(38, 18) | DoubleType | X
**DateType** | X    | X    | X    | X    | X    | X    | X    | X    | X    | StringType | DateType | TimestampType | X    | X    | X    | X    | X    | X    | X    | X
**TimestampType** | X    | X    | X    | X    | X    | X    | X    | X    | X    | StringType | DateType | TimestampType | X    | X    | X    | X    | X    | X    | X    | X
**ArrayType** | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | ArrayType* | X    | X    | X    | X    | X    | X    | X
**MapType** | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | MapType* | X    | X    | X    | X    | X    | X
**StructType** | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | StructType* | X    | X    | X    | X    | X
**NullType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | BinaryType | BooleanType | StringType | DateType | TimestampType | ArrayType | MapType | StructType | NullType | CalendarIntervalType | DecimalType(38, 18) | DoubleType | IntegerType
**CalendarIntervalType** | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | CalendarIntervalType | X    | X    | X
Note: ArrayType\*, MapType\*, StructType\* are castable only when the internal child types also match; otherwise, not castable
### How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15691 from gatorsmile/implicitTypeCasting.
2016-11-02 21:01:03 -07:00
Reynold Xin fd90541c35 [SPARK-18214][SQL] Simplify RuntimeReplaceable type coercion
## What changes were proposed in this pull request?
RuntimeReplaceable is used to create aliases for expressions, but the way it deals with type coercion is pretty weird (each expression is responsible for how to handle type coercion, which does not obey the normal implicit type cast rules).

This patch simplifies its handling by allowing the analyzer to traverse into the actual expression of a RuntimeReplaceable.

## How was this patch tested?
- Correctness should be guaranteed by existing unit tests already
- Removed SQLCompatibilityFunctionSuite and moved it sql-compatibility-functions.sql
- Added a new test case in sql-compatibility-functions.sql for verifying explain behavior.

Author: Reynold Xin <rxin@databricks.com>

Closes #15723 from rxin/SPARK-18214.
2016-11-02 15:53:02 -07:00
Xiangrui Meng 02f203107b [SPARK-14393][SQL] values generated by non-deterministic functions shouldn't change after coalesce or union
## What changes were proposed in this pull request?

When a user appended a column using a "nondeterministic" function to a DataFrame, e.g., `rand`, `randn`, and `monotonically_increasing_id`, the expected semantic is the following:
- The value in each row should remain unchanged, as if we materialize the column immediately, regardless of later DataFrame operations.

However, since we use `TaskContext.getPartitionId` to get the partition index from the current thread, the values from nondeterministic columns might change if we call `union` or `coalesce` after. `TaskContext.getPartitionId` returns the partition index of the current Spark task, which might not be the corresponding partition index of the DataFrame where we defined the column.

See the unit tests below or JIRA for examples.

This PR uses the partition index from `RDD.mapPartitionWithIndex` instead of `TaskContext` and fixes the partition initialization logic in whole-stage codegen, normal codegen, and codegen fallback. `initializeStatesForPartition(partitionIndex: Int)` was added to `Projection`, `Nondeterministic`, and `Predicate` (codegen) and initialized right after object creation in `mapPartitionWithIndex`. `newPredicate` now returns a `Predicate` instance rather than a function for proper initialization.
## How was this patch tested?

Unit tests. (Actually I'm not very confident that this PR fixed all issues without introducing new ones ...)

cc: rxin davies

Author: Xiangrui Meng <meng@databricks.com>

Closes #15567 from mengxr/SPARK-14393.
2016-11-02 11:41:49 -07:00
Takeshi YAMAMURO 4af0ce2d96 [SPARK-17683][SQL] Support ArrayType in Literal.apply
## What changes were proposed in this pull request?

This pr is to add pattern-matching entries for array data in `Literal.apply`.
## How was this patch tested?

Added tests in `LiteralExpressionSuite`.

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

Closes #15257 from maropu/SPARK-17683.
2016-11-02 11:29:26 -07:00
eyal farago f151bd1af8 [SPARK-16839][SQL] Simplify Struct creation code path
## What changes were proposed in this pull request?

Simplify struct creation, especially the aspect of `CleanupAliases` which missed some aliases when handling trees created by `CreateStruct`.

This PR includes:

1. A failing test (create struct with nested aliases, some of the aliases survive `CleanupAliases`).
2. A fix that transforms `CreateStruct` into a `CreateNamedStruct` constructor, effectively eliminating `CreateStruct` from all expression trees.
3. A `NamePlaceHolder` used by `CreateStruct` when column names cannot be extracted from unresolved `NamedExpression`.
4. A new Analyzer rule that resolves `NamePlaceHolder` into a string literal once the `NamedExpression` is resolved.
5. `CleanupAliases` code was simplified as it no longer has to deal with `CreateStruct`'s top level columns.

## How was this patch tested?
Running all tests-suits in package org.apache.spark.sql, especially including the analysis suite, making sure added test initially fails, after applying suggested fix rerun the entire analysis package successfully.

Modified few tests that expected `CreateStruct` which is now transformed into `CreateNamedStruct`.

Author: eyal farago <eyal farago>
Author: Herman van Hovell <hvanhovell@databricks.com>
Author: eyal farago <eyal.farago@gmail.com>
Author: Eyal Farago <eyal.farago@actimize.com>
Author: Hyukjin Kwon <gurwls223@gmail.com>
Author: eyalfa <eyal.farago@gmail.com>

Closes #15718 from hvanhovell/SPARK-16839-2.
2016-11-02 11:12:20 +01:00
Sean Owen 9c8deef64e
[SPARK-18076][CORE][SQL] Fix default Locale used in DateFormat, NumberFormat to Locale.US
## What changes were proposed in this pull request?

Fix `Locale.US` for all usages of `DateFormat`, `NumberFormat`
## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #15610 from srowen/SPARK-18076.
2016-11-02 09:39:15 +00:00
Eric Liang abefe2ec42 [SPARK-18183][SPARK-18184] Fix INSERT [INTO|OVERWRITE] TABLE ... PARTITION for Datasource tables
## What changes were proposed in this pull request?

There are a couple issues with the current 2.1 behavior when inserting into Datasource tables with partitions managed by Hive.

(1) OVERWRITE TABLE ... PARTITION will actually overwrite the entire table instead of just the specified partition.
(2) INSERT|OVERWRITE does not work with partitions that have custom locations.

This PR fixes both of these issues for Datasource tables managed by Hive. The behavior for legacy tables or when `manageFilesourcePartitions = false` is unchanged.

There is one other issue in that INSERT OVERWRITE with dynamic partitions will overwrite the entire table instead of just the updated partitions, but this behavior is pretty complicated to implement for Datasource tables. We should address that in a future release.

## How was this patch tested?

Unit tests.

Author: Eric Liang <ekl@databricks.com>

Closes #15705 from ericl/sc-4942.
2016-11-02 14:15:10 +08:00
hyukjinkwon 01dd008301 [SPARK-17764][SQL] Add to_json supporting to convert nested struct column to JSON string
## What changes were proposed in this pull request?

This PR proposes to add `to_json` function in contrast with `from_json` in Scala, Java and Python.

It'd be useful if we can convert a same column from/to json. Also, some datasources do not support nested types. If we are forced to save a dataframe into those data sources, we might be able to work around by this function.

The usage is as below:

``` scala
val df = Seq(Tuple1(Tuple1(1))).toDF("a")
df.select(to_json($"a").as("json")).show()
```

``` bash
+--------+
|    json|
+--------+
|{"_1":1}|
+--------+
```
## How was this patch tested?

Unit tests in `JsonFunctionsSuite` and `JsonExpressionsSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15354 from HyukjinKwon/SPARK-17764.
2016-11-01 12:46:41 -07:00
Herman van Hovell 0cba535af3 Revert "[SPARK-16839][SQL] redundant aliases after cleanupAliases"
This reverts commit 5441a6269e.
2016-11-01 17:30:37 +01:00
eyal farago 5441a6269e [SPARK-16839][SQL] redundant aliases after cleanupAliases
## What changes were proposed in this pull request?

Simplify struct creation, especially the aspect of `CleanupAliases` which missed some aliases when handling trees created by `CreateStruct`.

This PR includes:

1. A failing test (create struct with nested aliases, some of the aliases survive `CleanupAliases`).
2. A fix that transforms `CreateStruct` into a `CreateNamedStruct` constructor, effectively eliminating `CreateStruct` from all expression trees.
3. A `NamePlaceHolder` used by `CreateStruct` when column names cannot be extracted from unresolved `NamedExpression`.
4. A new Analyzer rule that resolves `NamePlaceHolder` into a string literal once the `NamedExpression` is resolved.
5. `CleanupAliases` code was simplified as it no longer has to deal with `CreateStruct`'s top level columns.

## How was this patch tested?

running all tests-suits in package org.apache.spark.sql, especially including the analysis suite, making sure added test initially fails, after applying suggested fix rerun the entire analysis package successfully.

modified few tests that expected `CreateStruct` which is now transformed into `CreateNamedStruct`.

Credit goes to hvanhovell for assisting with this PR.

Author: eyal farago <eyal farago>
Author: eyal farago <eyal.farago@gmail.com>
Author: Herman van Hovell <hvanhovell@databricks.com>
Author: Eyal Farago <eyal.farago@actimize.com>
Author: Hyukjin Kwon <gurwls223@gmail.com>
Author: eyalfa <eyal.farago@gmail.com>

Closes #14444 from eyalfa/SPARK-16839_redundant_aliases_after_cleanupAliases.
2016-11-01 17:12:20 +01:00
Eric Liang ccb1154304 [SPARK-17970][SQL] store partition spec in metastore for data source table
## What changes were proposed in this pull request?

We should follow hive table and also store partition spec in metastore for data source table.
This brings 2 benefits:

1. It's more flexible to manage the table data files, as users can use `ADD PARTITION`, `DROP PARTITION` and `RENAME PARTITION`
2. We don't need to cache all file status for data source table anymore.

## How was this patch tested?

existing tests.

Author: Eric Liang <ekl@databricks.com>
Author: Michael Allman <michael@videoamp.com>
Author: Eric Liang <ekhliang@gmail.com>
Author: Wenchen Fan <wenchen@databricks.com>

Closes #15515 from cloud-fan/partition.
2016-10-27 14:22:30 -07:00
jiangxingbo fa7d9d7082 [SPARK-18063][SQL] Failed to infer constraints over multiple aliases
## What changes were proposed in this pull request?

The `UnaryNode.getAliasedConstraints` function fails to replace all expressions by their alias where constraints contains more than one expression to be replaced.
For example:
```
val tr = LocalRelation('a.int, 'b.string, 'c.int)
val multiAlias = tr.where('a === 'c + 10).select('a.as('x), 'c.as('y))
multiAlias.analyze.constraints
```
currently outputs:
```
ExpressionSet(Seq(
    IsNotNull(resolveColumn(multiAlias.analyze, "x")),
    IsNotNull(resolveColumn(multiAlias.analyze, "y"))
)
```
The constraint `resolveColumn(multiAlias.analyze, "x") === resolveColumn(multiAlias.analyze, "y") + 10)` is missing.

## How was this patch tested?

Add new test cases in `ConstraintPropagationSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15597 from jiangxb1987/alias-constraints.
2016-10-26 20:12:20 +02:00
jiangxingbo 3c023570b2 [SPARK-17733][SQL] InferFiltersFromConstraints rule never terminates for query
## What changes were proposed in this pull request?

The function `QueryPlan.inferAdditionalConstraints` and `UnaryNode.getAliasedConstraints` can produce a non-converging set of constraints for recursive functions. For instance, if we have two constraints of the form(where a is an alias):
`a = b, a = f(b, c)`
Applying both these rules in the next iteration would infer:
`f(b, c) = f(f(b, c), c)`
This process repeated, the iteration won't converge and the set of constraints will grow larger and larger until OOM.

~~To fix this problem, we collect alias from expressions and skip infer constraints if we are to transform an `Expression` to another which contains it.~~
To fix this problem, we apply additional check in `inferAdditionalConstraints`, when it's possible to generate recursive constraints, we skip generate that.

## How was this patch tested?

Add new testcase in `SQLQuerySuite`/`InferFiltersFromConstraintsSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15319 from jiangxb1987/constraints.
2016-10-26 17:09:48 +02:00
CodingCat a81fba048f [SPARK-18058][SQL] Comparing column types ignoring Nullability in Union and SetOperation
## What changes were proposed in this pull request?

The PR tries to fix [SPARK-18058](https://issues.apache.org/jira/browse/SPARK-18058) which refers to a bug that the column types are compared with the extra care about Nullability in Union and SetOperation.

This PR converts the columns types by setting all fields as nullable before comparison

## How was this patch tested?

regular unit test cases

Author: CodingCat <zhunansjtu@gmail.com>

Closes #15595 from CodingCat/SPARK-18058.
2016-10-23 19:42:11 +02:00
Zheng RuiFeng a8ea4da8d0
[SPARK-17331][FOLLOWUP][ML][CORE] Avoid allocating 0-length arrays
## What changes were proposed in this pull request?

`Array[T]()` -> `Array.empty[T]` to avoid allocating 0-length arrays.
Use regex `find . -name '*.scala' | xargs -i bash -c 'egrep "Array\[[A-Za-z]+\]\(\)" -n {} && echo {}'` to find modification candidates.

cc srowen

## How was this patch tested?
existing tests

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #15564 from zhengruifeng/avoid_0_length_array.
2016-10-21 09:49:37 +01:00
Wenchen Fan 4329c5cea4 [SPARK-17873][SQL] ALTER TABLE RENAME TO should allow users to specify database in destination table name(but have to be same as source table)
## What changes were proposed in this pull request?

Unlike Hive, in Spark SQL, ALTER TABLE RENAME TO cannot move a table from one database to another(e.g. `ALTER TABLE db1.tbl RENAME TO db2.tbl2`), and will report error if the database in source table and destination table is different. So in #14955 , we forbid users to specify database of destination table in ALTER TABLE RENAME TO, to be consistent with other database systems and also make it easier to rename tables in non-current database, e.g. users can write `ALTER TABLE db1.tbl RENAME TO tbl2`, instead of `ALTER TABLE db1.tbl RENAME TO db1.tbl2`.

However, this is a breaking change. Users may already have queries that specify database of destination table in ALTER TABLE RENAME TO.

This PR reverts most of #14955 , and simplify the usage of ALTER TABLE RENAME TO by making database of source table the default database of destination table, instead of current database, so that users can still write `ALTER TABLE db1.tbl RENAME TO tbl2`, which is consistent with other databases like MySQL, Postgres, etc.

## How was this patch tested?

The added back tests and some new tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15434 from cloud-fan/revert.
2016-10-18 20:23:13 -07:00
Jakob Odersky 9dc0ca060d [SPARK-17368][SQL] Add support for value class serialization and deserialization
## What changes were proposed in this pull request?
Value classes were unsupported because catalyst data types were
obtained through reflection on erased types, which would resolve to a
value class' wrapped type and hence lead to unavailable methods during
code generation.

E.g. the following class
```scala
case class Foo(x: Int) extends AnyVal
```
would be seen as an `int` in catalyst and will cause instance cast failures when generated java code tries to treat it as a `Foo`.

This patch simply removes the erasure step when getting data types for
catalyst.

## How was this patch tested?
Additional tests in `ExpressionEncoderSuite`.

Author: Jakob Odersky <jakob@odersky.com>

Closes #15284 from jodersky/value-classes.
2016-10-13 17:48:09 -07:00
prigarg d5580ebaa0 [SPARK-17884][SQL] To resolve Null pointer exception when casting from empty string to interval type.
## What changes were proposed in this pull request?
This change adds a check in castToInterval method of Cast expression , such that if converted value is null , then isNull variable should be set to true.

Earlier, the expression Cast(Literal(), CalendarIntervalType) was throwing NullPointerException because of the above mentioned reason.

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

jira entry for detail: https://issues.apache.org/jira/browse/SPARK-17884

Author: prigarg <prigarg@adobe.com>

Closes #15449 from priyankagargnitk/SPARK-17884.
2016-10-12 10:14:45 -07:00
Liang-Chi Hsieh c8c090640a [SPARK-17821][SQL] Support And and Or in Expression Canonicalize
## What changes were proposed in this pull request?

Currently `Canonicalize` object doesn't support `And` and `Or`. So we can compare canonicalized form of predicates consistently. We should add the support.

## How was this patch tested?

Jenkins tests.

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

Closes #15388 from viirya/canonicalize-and-or.
2016-10-11 16:06:40 +08:00
jiangxingbo 16590030c1 [SPARK-17741][SQL] Grammar to parse top level and nested data fields separately
## What changes were proposed in this pull request?

Currently we use the same rule to parse top level and nested data fields. For example:
```
create table tbl_x(
  id bigint,
  nested struct<col1:string,col2:string>
)
```
Shows both syntaxes. In this PR we split this rule in a top-level and nested rule.

Before this PR,
```
sql("CREATE TABLE my_tab(column1: INT)")
```
works fine.
After this PR, it will throw a `ParseException`:
```
scala> sql("CREATE TABLE my_tab(column1: INT)")
org.apache.spark.sql.catalyst.parser.ParseException:
no viable alternative at input 'CREATE TABLE my_tab(column1:'(line 1, pos 27)
```

## How was this patch tested?
Add new testcases in `SparkSqlParserSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15346 from jiangxb1987/cdt.
2016-10-09 22:00:54 -07:00
jiangxingbo 26fbca4806 [SPARK-17832][SQL] TableIdentifier.quotedString creates un-parseable names when name contains a backtick
## What changes were proposed in this pull request?

The `quotedString` method in `TableIdentifier` and `FunctionIdentifier` produce an illegal (un-parseable) name when the name contains a backtick. For example:
```
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser._
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
val complexName = TableIdentifier("`weird`table`name", Some("`d`b`1"))
parseTableIdentifier(complexName.unquotedString) // Does not work
parseTableIdentifier(complexName.quotedString) // Does not work
parseExpression(complexName.unquotedString) // Does not work
parseExpression(complexName.quotedString) // Does not work
```
We should handle the backtick properly to make `quotedString` parseable.

## How was this patch tested?
Add new testcases in `TableIdentifierParserSuite` and `ExpressionParserSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15403 from jiangxb1987/backtick.
2016-10-09 21:52:46 -07:00
Herman van Hovell 97594c29b7 [SPARK-17761][SQL] Remove MutableRow
## What changes were proposed in this pull request?
In practice we cannot guarantee that an `InternalRow` is immutable. This makes the `MutableRow` almost redundant. This PR folds `MutableRow` into `InternalRow`.

The code below illustrates the immutability issue with InternalRow:
```scala
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
val struct = new GenericMutableRow(1)
val row = InternalRow(struct, 1)
println(row)
scala> [[null], 1]
struct.setInt(0, 42)
println(row)
scala> [[42], 1]
```

This might be somewhat controversial, so feedback is appreciated.

## How was this patch tested?
Existing tests.

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

Closes #15333 from hvanhovell/SPARK-17761.
2016-10-07 14:03:45 -07:00
Herman van Hovell 5fd54b994e [SPARK-17758][SQL] Last returns wrong result in case of empty partition
## What changes were proposed in this pull request?
The result of the `Last` function can be wrong when the last partition processed is empty. It can return `null` instead of the expected value. For example, this can happen when we process partitions in the following order:
```
- Partition 1 [Row1, Row2]
- Partition 2 [Row3]
- Partition 3 []
```
In this case the `Last` function will currently return a null, instead of the value of `Row3`.

This PR fixes this by adding a `valueSet` flag to the `Last` function.

## How was this patch tested?
We only used end to end tests for `DeclarativeAggregateFunction`s. I have added an evaluator for these functions so we can tests them in catalyst. I have added a `LastTestSuite` to test the `Last` aggregate function.

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

Closes #15348 from hvanhovell/SPARK-17758.
2016-10-05 16:05:30 -07:00
Herman van Hovell 89516c1c4a [SPARK-17258][SQL] Parse scientific decimal literals as decimals
## What changes were proposed in this pull request?
Currently Spark SQL parses regular decimal literals (e.g. `10.00`) as decimals and scientific decimal literals (e.g. `10.0e10`) as doubles. The difference between the two confuses most users. This PR unifies the parsing behavior and also parses scientific decimal literals as decimals.

This implications in tests are limited to a single Hive compatibility test.

## How was this patch tested?
Updated tests in `ExpressionParserSuite` and `SQLQueryTestSuite`.

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

Closes #14828 from hvanhovell/SPARK-17258.
2016-10-04 23:48:26 -07:00
Tejas Patil a99743d053 [SPARK-17495][SQL] Add Hash capability semantically equivalent to Hive's
## What changes were proposed in this pull request?

Jira : https://issues.apache.org/jira/browse/SPARK-17495

Spark internally uses Murmur3Hash for partitioning. This is different from the one used by Hive. For queries which use bucketing this leads to different results if one tries the same query on both engines. For us, we want users to have backward compatibility to that one can switch parts of applications across the engines without observing regressions.

This PR includes `HiveHash`, `HiveHashFunction`, `HiveHasher` which mimics Hive's hashing at https://github.com/apache/hive/blob/master/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java#L638

I am intentionally not introducing any usages of this hash function in rest of the code to keep this PR small. My eventual goal is to have Hive bucketing support in Spark. Once this PR gets in, I will make hash function pluggable in relevant areas (eg. `HashPartitioning`'s `partitionIdExpression` has Murmur3 hardcoded : https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala#L265)

## How was this patch tested?

Added `HiveHashSuite`

Author: Tejas Patil <tejasp@fb.com>

Closes #15047 from tejasapatil/SPARK-17495_hive_hash.
2016-10-04 18:59:31 -07:00
Takuya UESHIN b1b47274bf [SPARK-17702][SQL] Code generation including too many mutable states exceeds JVM size limit.
## What changes were proposed in this pull request?

Code generation including too many mutable states exceeds JVM size limit to extract values from `references` into fields in the constructor.
We should split the generated extractions in the constructor into smaller functions.

## How was this patch tested?

I added some tests to check if the generated codes for the expressions exceed or not.

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

Closes #15275 from ueshin/issues/SPARK-17702.
2016-10-03 21:48:58 -07:00
Herman van Hovell 2bbecdec20 [SPARK-17753][SQL] Allow a complex expression as the input a value based case statement
## What changes were proposed in this pull request?
We currently only allow relatively simple expressions as the input for a value based case statement. Expressions like `case (a > 1) or (b = 2) when true then 1 when false then 0 end` currently fail. This PR adds support for such expressions.

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

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

Closes #15322 from hvanhovell/SPARK-17753.
2016-10-03 19:32:59 -07:00
Dongjoon Hyun aef506e39a [SPARK-17739][SQL] Collapse adjacent similar Window operators
## What changes were proposed in this pull request?

Currently, Spark does not collapse adjacent windows with the same partitioning and sorting. This PR implements `CollapseWindow` optimizer to do the followings.

1. If the partition specs and order specs are the same, collapse into the parent.
2. If the partition specs are the same and one order spec is a prefix of the other, collapse to the more specific one.

For example:
```scala
val df = spark.range(1000).select($"id" % 100 as "grp", $"id", rand() as "col1", rand() as "col2")

// Add summary statistics for all columns
import org.apache.spark.sql.expressions.Window
val cols = Seq("id", "col1", "col2")
val window = Window.partitionBy($"grp").orderBy($"id")
val result = cols.foldLeft(df) { (base, name) =>
  base.withColumn(s"${name}_avg", avg(col(name)).over(window))
      .withColumn(s"${name}_stddev", stddev(col(name)).over(window))
      .withColumn(s"${name}_min", min(col(name)).over(window))
      .withColumn(s"${name}_max", max(col(name)).over(window))
}
```

**Before**
```scala
scala> result.explain
== Physical Plan ==
Window [max(col2#19) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_max#234], [grp#17L], [id#14L ASC NULLS FIRST]
+- Window [min(col2#19) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_min#216], [grp#17L], [id#14L ASC NULLS FIRST]
   +- Window [stddev_samp(col2#19) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_stddev#191], [grp#17L], [id#14L ASC NULLS FIRST]
      +- Window [avg(col2#19) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_avg#167], [grp#17L], [id#14L ASC NULLS FIRST]
         +- Window [max(col1#18) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_max#152], [grp#17L], [id#14L ASC NULLS FIRST]
            +- Window [min(col1#18) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_min#138], [grp#17L], [id#14L ASC NULLS FIRST]
               +- Window [stddev_samp(col1#18) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_stddev#117], [grp#17L], [id#14L ASC NULLS FIRST]
                  +- Window [avg(col1#18) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_avg#97], [grp#17L], [id#14L ASC NULLS FIRST]
                     +- Window [max(id#14L) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_max#86L], [grp#17L], [id#14L ASC NULLS FIRST]
                        +- Window [min(id#14L) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_min#76L], [grp#17L], [id#14L ASC NULLS FIRST]
                           +- *Project [grp#17L, id#14L, col1#18, col2#19, id_avg#26, id_stddev#42]
                              +- Window [stddev_samp(_w0#59) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_stddev#42], [grp#17L], [id#14L ASC NULLS FIRST]
                                 +- *Project [grp#17L, id#14L, col1#18, col2#19, id_avg#26, cast(id#14L as double) AS _w0#59]
                                    +- Window [avg(id#14L) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_avg#26], [grp#17L], [id#14L ASC NULLS FIRST]
                                       +- *Sort [grp#17L ASC NULLS FIRST, id#14L ASC NULLS FIRST], false, 0
                                          +- Exchange hashpartitioning(grp#17L, 200)
                                             +- *Project [(id#14L % 100) AS grp#17L, id#14L, rand(-6329949029880411066) AS col1#18, rand(-7251358484380073081) AS col2#19]
                                                +- *Range (0, 1000, step=1, splits=Some(8))
```

**After**
```scala
scala> result.explain
== Physical Plan ==
Window [max(col2#5) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_max#220, min(col2#5) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_min#202, stddev_samp(col2#5) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_stddev#177, avg(col2#5) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_avg#153, max(col1#4) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_max#138, min(col1#4) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_min#124, stddev_samp(col1#4) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_stddev#103, avg(col1#4) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_avg#83, max(id#0L) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_max#72L, min(id#0L) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_min#62L], [grp#3L], [id#0L ASC NULLS FIRST]
+- *Project [grp#3L, id#0L, col1#4, col2#5, id_avg#12, id_stddev#28]
   +- Window [stddev_samp(_w0#45) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_stddev#28], [grp#3L], [id#0L ASC NULLS FIRST]
      +- *Project [grp#3L, id#0L, col1#4, col2#5, id_avg#12, cast(id#0L as double) AS _w0#45]
         +- Window [avg(id#0L) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_avg#12], [grp#3L], [id#0L ASC NULLS FIRST]
            +- *Sort [grp#3L ASC NULLS FIRST, id#0L ASC NULLS FIRST], false, 0
               +- Exchange hashpartitioning(grp#3L, 200)
                  +- *Project [(id#0L % 100) AS grp#3L, id#0L, rand(6537478539664068821) AS col1#4, rand(-8961093871295252795) AS col2#5]
                     +- *Range (0, 1000, step=1, splits=Some(8))
```

## How was this patch tested?

Pass the Jenkins tests with a newly added testsuite.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15317 from dongjoon-hyun/SPARK-17739.
2016-09-30 21:05:06 -07:00
Liang-Chi Hsieh 566d7f2827 [SPARK-17653][SQL] Remove unnecessary distincts in multiple unions
## What changes were proposed in this pull request?

Currently for `Union [Distinct]`, a `Distinct` operator is necessary to be on the top of `Union`. Once there are adjacent `Union [Distinct]`,  there will be multiple `Distinct` in the query plan.

E.g.,

For a query like: select 1 a union select 2 b union select 3 c

Before this patch, its physical plan looks like:

    *HashAggregate(keys=[a#13], functions=[])
    +- Exchange hashpartitioning(a#13, 200)
       +- *HashAggregate(keys=[a#13], functions=[])
          +- Union
             :- *HashAggregate(keys=[a#13], functions=[])
             :  +- Exchange hashpartitioning(a#13, 200)
             :     +- *HashAggregate(keys=[a#13], functions=[])
             :        +- Union
             :           :- *Project [1 AS a#13]
             :           :  +- Scan OneRowRelation[]
             :           +- *Project [2 AS b#14]
             :              +- Scan OneRowRelation[]
             +- *Project [3 AS c#15]
                +- Scan OneRowRelation[]

Only the top distinct should be necessary.

After this patch, the physical plan looks like:

    *HashAggregate(keys=[a#221], functions=[], output=[a#221])
    +- Exchange hashpartitioning(a#221, 5)
       +- *HashAggregate(keys=[a#221], functions=[], output=[a#221])
          +- Union
             :- *Project [1 AS a#221]
             :  +- Scan OneRowRelation[]
             :- *Project [2 AS b#222]
             :  +- Scan OneRowRelation[]
             +- *Project [3 AS c#223]
                +- Scan OneRowRelation[]

## How was this patch tested?

Jenkins tests.

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

Closes #15238 from viirya/remove-extra-distinct-union.
2016-09-29 14:30:23 -07:00
Michael Armbrust fe33121a53 [SPARK-17699] Support for parsing JSON string columns
Spark SQL has great support for reading text files that contain JSON data.  However, in many cases the JSON data is just one column amongst others.  This is particularly true when reading from sources such as Kafka.  This PR adds a new functions `from_json` that converts a string column into a nested `StructType` with a user specified schema.

Example usage:
```scala
val df = Seq("""{"a": 1}""").toDS()
val schema = new StructType().add("a", IntegerType)

df.select(from_json($"value", schema) as 'json) // => [json: <a: int>]
```

This PR adds support for java, scala and python.  I leveraged our existing JSON parsing support by moving it into catalyst (so that we could define expressions using it).  I left SQL out for now, because I'm not sure how users would specify a schema.

Author: Michael Armbrust <michael@databricks.com>

Closes #15274 from marmbrus/jsonParser.
2016-09-29 13:01:10 -07:00
Josh Rosen 37eb9184f1 [SPARK-17712][SQL] Fix invalid pushdown of data-independent filters beneath aggregates
## What changes were proposed in this pull request?

This patch fixes a minor correctness issue impacting the pushdown of filters beneath aggregates. Specifically, if a filter condition references no grouping or aggregate columns (e.g. `WHERE false`) then it would be incorrectly pushed beneath an aggregate.

Intuitively, the only case where you can push a filter beneath an aggregate is when that filter is deterministic and is defined over the grouping columns / expressions, since in that case the filter is acting to exclude entire groups from the query (like a `HAVING` clause). The existing code would only push deterministic filters beneath aggregates when all of the filter's references were grouping columns, but this logic missed the case where a filter has no references. For example, `WHERE false` is deterministic but is independent of the actual data.

This patch fixes this minor bug by adding a new check to ensure that we don't push filters beneath aggregates when those filters don't reference any columns.

## How was this patch tested?

New regression test in FilterPushdownSuite.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #15289 from JoshRosen/SPARK-17712.
2016-09-28 19:03:05 -07:00
Kazuaki Ishizaki 85b0a15754 [SPARK-15962][SQL] Introduce implementation with a dense format for UnsafeArrayData
## What changes were proposed in this pull request?

This PR introduces more compact representation for ```UnsafeArrayData```.

```UnsafeArrayData``` needs to accept ```null``` value in each entry of an array. In the current version, it has three parts
```
[numElements] [offsets] [values]
```
`Offsets` has the number of `numElements`, and represents `null` if its value is negative. It may increase memory footprint, and introduces an indirection for accessing each of `values`.

This PR uses bitvectors to represent nullability for each element like `UnsafeRow`, and eliminates an indirection for accessing each element. The new ```UnsafeArrayData``` has four parts.
```
[numElements][null bits][values or offset&length][variable length portion]
```
In the `null bits` region, we store 1 bit per element, represents whether an element is null. Its total size is ceil(numElements / 8) bytes, and it is aligned to 8-byte boundaries.
In the `values or offset&length` region, we store the content of elements. For fields that hold fixed-length primitive types, such as long, double, or int, we store the value directly in the field. For fields with non-primitive or variable-length values, we store a relative offset (w.r.t. the base address of the array) that points to the beginning of the variable-length field and length (they are combined into a long). Each is word-aligned. For `variable length portion`, each is aligned to 8-byte boundaries.

The new format can reduce memory footprint and improve performance of accessing each element. An example of memory foot comparison:
1024x1024 elements integer array
Size of ```baseObject``` for ```UnsafeArrayData```: 8 + 1024x1024 + 1024x1024 = 2M bytes
Size of ```baseObject``` for ```UnsafeArrayData```: 8 + 1024x1024/8 + 1024x1024 = 1.25M bytes

In summary, we got 1.0-2.6x performance improvements over the code before applying this PR.
Here are performance results of [benchmark programs](04d2e4b6db/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala):

**Read UnsafeArrayData**: 1.7x and 1.6x performance improvements over the code before applying this PR
````
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)

Without SPARK-15962
Read UnsafeArrayData:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            430 /  436        390.0           2.6       1.0X
Double                                         456 /  485        367.8           2.7       0.9X

With SPARK-15962
Read UnsafeArrayData:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            252 /  260        666.1           1.5       1.0X
Double                                         281 /  292        597.7           1.7       0.9X
````
**Write UnsafeArrayData**: 1.0x and 1.1x performance improvements over the code before applying this PR
````
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)

Without SPARK-15962
Write UnsafeArrayData:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            203 /  273        103.4           9.7       1.0X
Double                                         239 /  356         87.9          11.4       0.8X

With SPARK-15962
Write UnsafeArrayData:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            196 /  249        107.0           9.3       1.0X
Double                                         227 /  367         92.3          10.8       0.9X
````

**Get primitive array from UnsafeArrayData**: 2.6x and 1.6x performance improvements over the code before applying this PR
````
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)

Without SPARK-15962
Get primitive array from UnsafeArrayData: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            207 /  217        304.2           3.3       1.0X
Double                                         257 /  363        245.2           4.1       0.8X

With SPARK-15962
Get primitive array from UnsafeArrayData: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            151 /  198        415.8           2.4       1.0X
Double                                         214 /  394        293.6           3.4       0.7X
````

**Create UnsafeArrayData from primitive array**: 1.7x and 2.1x performance improvements over the code before applying this PR
````
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)

Without SPARK-15962
Create UnsafeArrayData from primitive array: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            340 /  385        185.1           5.4       1.0X
Double                                         479 /  705        131.3           7.6       0.7X

With SPARK-15962
Create UnsafeArrayData from primitive array: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            206 /  211        306.0           3.3       1.0X
Double                                         232 /  406        271.6           3.7       0.9X
````

1.7x and 1.4x performance improvements in [```UDTSerializationBenchmark```](https://github.com/apache/spark/blob/master/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala)  over the code before applying this PR
````
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)

Without SPARK-15962
VectorUDT de/serialization:              Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
serialize                                      442 /  533          0.0      441927.1       1.0X
deserialize                                    217 /  274          0.0      217087.6       2.0X

With SPARK-15962
VectorUDT de/serialization:              Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
serialize                                      265 /  318          0.0      265138.5       1.0X
deserialize                                    155 /  197          0.0      154611.4       1.7X
````

## How was this patch tested?

Added unit tests into ```UnsafeArraySuite```

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

Closes #13680 from kiszk/SPARK-15962.
2016-09-27 14:18:32 +08:00
Herman van Hovell 0d63487502 [SPARK-17616][SQL] Support a single distinct aggregate combined with a non-partial aggregate
## What changes were proposed in this pull request?
We currently cannot execute an aggregate that contains a single distinct aggregate function and an one or more non-partially plannable aggregate functions, for example:
```sql
select   grp,
         collect_list(col1),
         count(distinct col2)
from     tbl_a
group by 1
```
This is a regression from Spark 1.6. This is caused by the fact that the single distinct aggregation code path assumes that all aggregates can be planned in two phases (is partially aggregatable). This PR works around this issue by triggering the `RewriteDistinctAggregates` in such cases (this is similar to the approach taken in 1.6).

## How was this patch tested?
Created `RewriteDistinctAggregatesSuite` which checks if the aggregates with distinct aggregate functions get rewritten into two `Aggregates` and an `Expand`. Added a regression test to `DataFrameAggregateSuite`.

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

Closes #15187 from hvanhovell/SPARK-17616.
2016-09-22 14:29:27 -07:00
Wenchen Fan b50b34f561 [SPARK-17609][SQL] SessionCatalog.tableExists should not check temp view
## What changes were proposed in this pull request?

After #15054 , there is no place in Spark SQL that need `SessionCatalog.tableExists` to check temp views, so this PR makes `SessionCatalog.tableExists` only check permanent table/view and removes some hacks.

This PR also improves the `getTempViewOrPermanentTableMetadata` that is introduced in  #15054 , to make the code simpler.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15160 from cloud-fan/exists.
2016-09-22 12:52:09 +08:00
Davies Liu 8bde03bf9a [SPARK-17494][SQL] changePrecision() on compact decimal should respect rounding mode
## What changes were proposed in this pull request?

Floor()/Ceil() of decimal is implemented using changePrecision() by passing a rounding mode, but the rounding mode is not respected when the decimal is in compact mode (could fit within a Long).

This Update the changePrecision() to respect rounding mode, which could be ROUND_FLOOR, ROUND_CEIL, ROUND_HALF_UP, ROUND_HALF_EVEN.

## How was this patch tested?

Added regression tests.

Author: Davies Liu <davies@databricks.com>

Closes #15154 from davies/decimal_round.
2016-09-21 21:02:30 -07:00
Sean Zhong 3977223a32 [SPARK-17617][SQL] Remainder(%) expression.eval returns incorrect result on double value
## What changes were proposed in this pull request?

Remainder(%) expression's `eval()` returns incorrect result when the dividend is a big double. The reason is that Remainder converts the double dividend to decimal to do "%", and that lose precision.

This bug only affects the `eval()` that is used by constant folding, the codegen path is not impacted.

### Before change
```
scala> -5083676433652386516D % 10
res2: Double = -6.0

scala> spark.sql("select -5083676433652386516D % 10 as a").show
+---+
|  a|
+---+
|0.0|
+---+
```

### After change
```
scala> spark.sql("select -5083676433652386516D % 10 as a").show
+----+
|   a|
+----+
|-6.0|
+----+
```

## How was this patch tested?

Unit test.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #15171 from clockfly/SPARK-17617.
2016-09-21 16:53:34 +08:00