Commit graph

1348 commits

Author SHA1 Message Date
Kazuaki Ishizaki 2ce37b50fc [SPARK-23546][SQL] Refactor stateless methods/values in CodegenContext
## What changes were proposed in this pull request?

A current `CodegenContext` class has immutable value or method without mutable state, too.
This refactoring moves them to `CodeGenerator` object class which can be accessed from anywhere without an instantiated `CodegenContext` in the program.

## How was this patch tested?

Existing tests

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

Closes #20700 from kiszk/SPARK-23546.
2018-03-05 11:39:01 +01:00
KaiXinXiaoLei cdcccd7b41 [SPARK-23405] Generate additional constraints for Join's children
## What changes were proposed in this pull request?

(Please fill in changes proposed in this fix)
I run a sql: `select ls.cs_order_number from ls left semi join catalog_sales cs on ls.cs_order_number = cs.cs_order_number`, The `ls` table is a small table ,and the number is one. The `catalog_sales` table is a big table,  and the number is 10 billion. The task will be hang up. And i find the many null values of `cs_order_number` in the `catalog_sales` table. I think the null value should be removed in the logical plan.

>== Optimized Logical Plan ==
>Join LeftSemi, (cs_order_number#1 = cs_order_number#22)
>:- Project cs_order_number#1
>   : +- Filter isnotnull(cs_order_number#1)
>      : +- MetastoreRelation 100t, ls
>+- Project cs_order_number#22
>   +- MetastoreRelation 100t, catalog_sales

Now, use this patch, the plan will be:
>== Optimized Logical Plan ==
>Join LeftSemi, (cs_order_number#1 = cs_order_number#22)
>:- Project cs_order_number#1
>   : +- Filter isnotnull(cs_order_number#1)
>      : +- MetastoreRelation 100t, ls
>+- Project cs_order_number#22
>   : **+- Filter isnotnull(cs_order_number#22)**
>     :+- MetastoreRelation 100t, catalog_sales

## 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: KaiXinXiaoLei <584620569@qq.com>
Author: hanghang <584620569@qq.com>

Closes #20670 from KaiXinXiaoLei/Spark-23405.
2018-03-02 00:09:44 +08:00
Juliusz Sompolski 8077bb04f3 [SPARK-23445] ColumnStat refactoring
## What changes were proposed in this pull request?

Refactor ColumnStat to be more flexible.

* Split `ColumnStat` and `CatalogColumnStat` just like `CatalogStatistics` is split from `Statistics`. This detaches how the statistics are stored from how they are processed in the query plan. `CatalogColumnStat` keeps `min` and `max` as `String`, making it not depend on dataType information.
* For `CatalogColumnStat`, parse column names from property names in the metastore (`KEY_VERSION` property), not from metastore schema. This means that `CatalogColumnStat`s can be created for columns even if the schema itself is not stored in the metastore.
* Make all fields optional. `min`, `max` and `histogram` for columns were optional already. Having them all optional is more consistent, and gives flexibility to e.g. drop some of the fields through transformations if they are difficult / impossible to calculate.

The added flexibility will make it possible to have alternative implementations for stats, and separates stats collection from stats and estimation processing in plans.

## How was this patch tested?

Refactored existing tests to work with refactored `ColumnStat` and `CatalogColumnStat`.
New tests added in `StatisticsSuite` checking that backwards / forwards compatibility is not broken.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #20624 from juliuszsompolski/SPARK-23445.
2018-02-26 23:37:31 -08:00
hyukjinkwon ed86476098 [SPARK-23359][SQL] Adds an alias 'names' of 'fieldNames' in Scala's StructType
## What changes were proposed in this pull request?

This PR proposes to add an alias 'names' of  'fieldNames' in Scala. Please see the discussion in [SPARK-20090](https://issues.apache.org/jira/browse/SPARK-20090).

## How was this patch tested?

Unit tests added in `DataTypeSuite.scala`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20545 from HyukjinKwon/SPARK-23359.
2018-02-15 17:13:05 +08:00
caoxuewen 63b49fa2e5 [SPARK-23311][SQL][TEST] add FilterFunction test case for test CombineTypedFilters
## What changes were proposed in this pull request?

In the current test case for CombineTypedFilters, we lack the test of FilterFunction, so let's add it.
In addition, in TypedFilterOptimizationSuite's existing test cases, Let's extract a common LocalRelation.

## How was this patch tested?

add new test cases.

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

Closes #20482 from heary-cao/TypedFilterOptimizationSuite.
2018-02-03 00:02:03 -08:00
gatorsmile ca04c3ff23 [SPARK-23274][SQL] Fix ReplaceExceptWithFilter when the right's Filter contains the references that are not in the left output
## What changes were proposed in this pull request?
This PR is to fix the `ReplaceExceptWithFilter` rule when the right's Filter contains the references that are not in the left output.

Before this PR, we got the error like
```
java.util.NoSuchElementException: key not found: a
  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)
```

After this PR, `ReplaceExceptWithFilter ` will not take an effect in this case.

## How was this patch tested?
Added tests

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20444 from gatorsmile/fixReplaceExceptWithFilter.
2018-01-30 20:05:57 -08:00
Herman van Hovell 2d903cf9d3 [SPARK-23223][SQL] Make stacking dataset transforms more performant
## What changes were proposed in this pull request?
It is a common pattern to apply multiple transforms to a `Dataset` (using `Dataset.withColumn` for example. This is currently quite expensive because we run `CheckAnalysis` on the full plan and create an encoder for each intermediate `Dataset`.

This PR extends the usage of the `AnalysisBarrier` to include `CheckAnalysis`. By doing this we hide the already analyzed plan  from `CheckAnalysis` because barrier is a `LeafNode`. The `AnalysisBarrier` is in the `FinishAnalysis` phase of the optimizer.

We also make binding the `Dataset` encoder lazy. The bound encoder is only needed when we materialize the dataset.

## How was this patch tested?
Existing test should cover this.

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

Closes #20402 from hvanhovell/SPARK-23223.
2018-01-29 09:00:54 -08:00
caoxuewen 54dd7cf4ef [SPARK-23199][SQL] improved Removes repetition from group expressions in Aggregate
## What changes were proposed in this pull request?

Currently, all Aggregate operations will go into RemoveRepetitionFromGroupExpressions, but there is no group expression or there is no duplicate group expression in group expression, we not need copy for logic plan.

## How was this patch tested?

the existed test case.

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

Closes #20375 from heary-cao/RepetitionGroupExpressions.
2018-01-29 08:56:42 -08:00
Herman van Hovell e29b08add9 [SPARK-23208][SQL] Fix code generation for complex create array (related) expressions
## What changes were proposed in this pull request?
The `GenArrayData.genCodeToCreateArrayData` produces illegal java code when code splitting is enabled. This is used in `CreateArray` and `CreateMap` expressions for complex object arrays.

This issue is caused by a typo.

## How was this patch tested?
Added a regression test in `complexTypesSuite`.

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

Closes #20391 from hvanhovell/SPARK-23208.
2018-01-25 16:40:41 +08:00
caoxuewen 6f0ba8472d [MINOR][SQL] add new unit test to LimitPushdown
## What changes were proposed in this pull request?

This PR is repaired as follows
1、update y -> x in "left outer join" test case ,maybe is mistake.
2、add a new test case:"left outer join and left sides are limited"
3、add a new test case:"left outer join and right sides are limited"
4、add a new test case: "right outer join and right sides are limited"
5、add a new test case: "right outer join and left sides are limited"
6、Remove annotations without code implementation

## How was this patch tested?

add new unit test case.

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

Closes #20381 from heary-cao/LimitPushdownSuite.
2018-01-24 13:06:09 -08:00
Jacek Laskowski 76b8b840dd [MINOR] Typo fixes
## What changes were proposed in this pull request?

Typo fixes

## How was this patch tested?

Local build / Doc-only changes

Author: Jacek Laskowski <jacek@japila.pl>

Closes #20344 from jaceklaskowski/typo-fixes.
2018-01-22 13:55:14 -06:00
Marco Gaido e28eb43114 [SPARK-22036][SQL] Decimal multiplication with high precision/scale often returns NULL
## What changes were proposed in this pull request?

When there is an operation between Decimals and the result is a number which is not representable exactly with the result's precision and scale, Spark is returning `NULL`. This was done to reflect Hive's behavior, but it is against SQL ANSI 2011, which states that "If the result cannot be represented exactly in the result type, then whether it is rounded or truncated is implementation-defined". Moreover, Hive now changed its behavior in order to respect the standard, thanks to HIVE-15331.

Therefore, the PR propose to:
 - update the rules to determine the result precision and scale according to the new Hive's ones introduces in HIVE-15331;
 - round the result of the operations, when it is not representable exactly with the result's precision and scale, instead of returning `NULL`
 - introduce a new config `spark.sql.decimalOperations.allowPrecisionLoss` which default to `true` (ie. the new behavior) in order to allow users to switch back to the previous one.

Hive behavior reflects SQLServer's one. The only difference is that the precision and scale are adjusted for all the arithmetic operations in Hive, while SQL Server is said to do so only for multiplications and divisions in the documentation. This PR follows Hive's behavior.

A more detailed explanation is available here: https://mail-archives.apache.org/mod_mbox/spark-dev/201712.mbox/%3CCAEorWNAJ4TxJR9NBcgSFMD_VxTg8qVxusjP%2BAJP-x%2BJV9zH-yA%40mail.gmail.com%3E.

## How was this patch tested?

modified and added UTs. Comparisons with results of Hive and SQLServer.

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20023 from mgaido91/SPARK-22036.
2018-01-18 21:24:39 +08:00
Wang Gengliang 8598a982b4 [SPARK-23079][SQL] Fix query constraints propagation with aliases
## What changes were proposed in this pull request?

Previously, PR #19201 fix the problem of non-converging constraints.
After that PR #19149 improve the loop and constraints is inferred only once.
So the problem of non-converging constraints is gone.

However, the case below will fail.

```

spark.range(5).write.saveAsTable("t")
val t = spark.read.table("t")
val left = t.withColumn("xid", $"id" + lit(1)).as("x")
val right = t.withColumnRenamed("id", "xid").as("y")
val df = left.join(right, "xid").filter("id = 3").toDF()
checkAnswer(df, Row(4, 3))

```

Because `aliasMap` replace all the aliased child. See the test case in PR for details.

This PR is to fix this bug by removing useless code for preventing non-converging constraints.
It can be also fixed with #20270, but this is much simpler and clean up the code.

## How was this patch tested?

Unit test

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #20278 from gengliangwang/FixConstraintSimple.
2018-01-18 00:05:26 +08:00
Gabor Somogyi a9b845ebb5 [SPARK-22361][SQL][TEST] Add unit test for Window Frames
## What changes were proposed in this pull request?

There are already quite a few integration tests using window frames, but the unit tests coverage is not ideal.

In this PR the already existing tests are reorganized, extended and where gaps found additional cases added.

## How was this patch tested?

Automated: Pass the Jenkins.

Author: Gabor Somogyi <gabor.g.somogyi@gmail.com>

Closes #20019 from gaborgsomogyi/SPARK-22361.
2018-01-17 10:03:25 +08:00
xubo245 6c81fe227a [SPARK-23035][SQL] Fix improper information of TempTableAlreadyExistsException
## What changes were proposed in this pull request?

Problem: it throw TempTableAlreadyExistsException and output "Temporary table '$table' already exists" when we create temp view by using org.apache.spark.sql.catalyst.catalog.GlobalTempViewManager#create, it's improper.

So fix improper information about TempTableAlreadyExistsException when create temp view:

change "Temporary table"  to  "Temporary view"

## How was this patch tested?

test("rename temporary view - destination table already exists, with: CREATE TEMPORARY view")

test("rename temporary view - destination table with database name,with:CREATE TEMPORARY view")

Author: xubo245 <601450868@qq.com>

Closes #20227 from xubo245/fixDeprecated.
2018-01-15 23:13:15 +08:00
Marco Gaido 5050868069 [SPARK-23025][SQL] Support Null type in scala reflection
## What changes were proposed in this pull request?

Add support for `Null` type in the `schemaFor` method for Scala reflection.

## How was this patch tested?

Added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20219 from mgaido91/SPARK-23025.
2018-01-12 18:04:44 +08:00
Feng Liu 9b33dfc408 [SPARK-22951][SQL] fix aggregation after dropDuplicates on empty data frames
## What changes were proposed in this pull request?

(courtesy of liancheng)

Spark SQL supports both global aggregation and grouping aggregation. Global aggregation always return a single row with the initial aggregation state as the output, even there are zero input rows. Spark implements this by simply checking the number of grouping keys and treats an aggregation as a global aggregation if it has zero grouping keys.

However, this simple principle drops the ball in the following case:

```scala
spark.emptyDataFrame.dropDuplicates().agg(count($"*") as "c").show()
// +---+
// | c |
// +---+
// | 1 |
// +---+
```

The reason is that:

1. `df.dropDuplicates()` is roughly translated into something equivalent to:

```scala
val allColumns = df.columns.map { col }
df.groupBy(allColumns: _*).agg(allColumns.head, allColumns.tail: _*)
```

This translation is implemented in the rule `ReplaceDeduplicateWithAggregate`.

2. `spark.emptyDataFrame` contains zero columns and zero rows.

Therefore, rule `ReplaceDeduplicateWithAggregate` makes a confusing transformation roughly equivalent to the following one:

```scala
spark.emptyDataFrame.dropDuplicates()
=> spark.emptyDataFrame.groupBy().agg(Map.empty[String, String])
```

The above transformation is confusing because the resulting aggregate operator contains no grouping keys (because `emptyDataFrame` contains no columns), and gets recognized as a global aggregation. As a result, Spark SQL allocates a single row filled by the initial aggregation state and uses it as the output, and returns a wrong result.

To fix this issue, this PR tweaks `ReplaceDeduplicateWithAggregate` by appending a literal `1` to the grouping key list of the resulting `Aggregate` operator when the input plan contains zero output columns. In this way, `spark.emptyDataFrame.dropDuplicates()` is now translated into a grouping aggregation, roughly depicted as:

```scala
spark.emptyDataFrame.dropDuplicates()
=> spark.emptyDataFrame.groupBy(lit(1)).agg(Map.empty[String, String])
```

Which is now properly treated as a grouping aggregation and returns the correct answer.

## How was this patch tested?

New unit tests added

Author: Feng Liu <fengliu@databricks.com>

Closes #20174 from liufengdb/fix-duplicate.
2018-01-10 14:25:04 -08:00
Takeshi Yamamuro 2250cb75b9 [SPARK-22981][SQL] Fix incorrect results of Casting Struct to String
## What changes were proposed in this pull request?
This pr fixed the issue when casting structs into strings;
```
scala> val df = Seq(((1, "a"), 0), ((2, "b"), 0)).toDF("a", "b")
scala> df.write.saveAsTable("t")
scala> sql("SELECT CAST(a AS STRING) FROM t").show
+-------------------+
|                  a|
+-------------------+
|[0,1,1800000001,61]|
|[0,2,1800000001,62]|
+-------------------+
```
This pr modified the result into;
```
+------+
|     a|
+------+
|[1, a]|
|[2, b]|
+------+
```

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20176 from maropu/SPARK-22981.
2018-01-09 21:58:55 +08:00
Josh Rosen f20131dd35 [SPARK-22984] Fix incorrect bitmap copying and offset adjustment in GenerateUnsafeRowJoiner
## What changes were proposed in this pull request?

This PR fixes a longstanding correctness bug in `GenerateUnsafeRowJoiner`. This class was introduced in https://github.com/apache/spark/pull/7821 (July 2015 / Spark 1.5.0+) and is used to combine pairs of UnsafeRows in TungstenAggregationIterator, CartesianProductExec, and AppendColumns.

### Bugs fixed by this patch

1. **Incorrect combining of null-tracking bitmaps**: when concatenating two UnsafeRows, the implementation "Concatenate the two bitsets together into a single one, taking padding into account". If one row has no columns then it has a bitset size of 0, but the code was incorrectly assuming that if the left row had a non-zero number of fields then the right row would also have at least one field, so it was copying invalid bytes and and treating them as part of the bitset. I'm not sure whether this bug was also present in the original implementation or whether it was introduced in https://github.com/apache/spark/pull/7892 (which fixed another bug in this code).
2. **Incorrect updating of data offsets for null variable-length fields**: after updating the bitsets and copying fixed-length and variable-length data, we need to perform adjustments to the offsets pointing the start of variable length fields's data. The existing code was _conditionally_ adding a fixed offset to correct for the new length of the combined row, but it is unsafe to do this if the variable-length field has a null value: we always represent nulls by storing `0` in the fixed-length slot, but this code was incorrectly incrementing those values. This bug was present since the original version of `GenerateUnsafeRowJoiner`.

### Why this bug remained latent for so long

The PR which introduced `GenerateUnsafeRowJoiner` features several randomized tests, including tests of the cases where one side of the join has no fields and where string-valued fields are null. However, the existing assertions were too weak to uncover this bug:

- If a null field has a non-zero value in its fixed-length data slot then this will not cause problems for field accesses because the null-tracking bitmap should still be correct and we will not try to use the incorrect offset for anything.
- If the null tracking bitmap is corrupted by joining against a row with no fields then the corruption occurs in field numbers past the actual field numbers contained in the row. Thus valid `isNullAt()` calls will not read the incorrectly-set bits.

The existing `GenerateUnsafeRowJoinerSuite` tests only exercised `.get()` and `isNullAt()`, but didn't actually check the UnsafeRows for bit-for-bit equality, preventing these bugs from failing assertions. It turns out that there was even a [GenerateUnsafeRowJoinerBitsetSuite](03377d2522/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoinerBitsetSuite.scala) but it looks like it also didn't catch this problem because it only tested the bitsets in an end-to-end fashion by accessing them through the `UnsafeRow` interface instead of actually comparing the bitsets' bytes.

### Impact of these bugs

- This bug will cause `equals()` and `hashCode()` to be incorrect for these rows, which will be problematic in case`GenerateUnsafeRowJoiner`'s results are used as join or grouping keys.
- Chained / repeated invocations of `GenerateUnsafeRowJoiner` may result in reads from invalid null bitmap positions causing fields to incorrectly become NULL (see the end-to-end example below).
  - It looks like this generally only happens in `CartesianProductExec`, which our query optimizer often avoids executing (usually we try to plan a `BroadcastNestedLoopJoin` instead).

### End-to-end test case demonstrating the problem

The following query demonstrates how this bug may result in incorrect query results:

```sql
set spark.sql.autoBroadcastJoinThreshold=-1; -- Needed to trigger CartesianProductExec

create table a as select * from values 1;
create table b as select * from values 2;

SELECT
  t3.col1,
  t1.col1
FROM a t1
CROSS JOIN b t2
CROSS JOIN b t3
```

This should return `(2, 1)` but instead was returning `(null, 1)`.

Column pruning ends up trimming off all columns from `t2`, so when `t2` joins with another table this triggers the bitmap-copying bug. This incorrect bitmap is subsequently copied again when performing the final join, causing the final output to have an incorrectly-set null bit for the first field.

## How was this patch tested?

Strengthened the assertions in existing tests in GenerateUnsafeRowJoinerSuite. Also verified that the end-to-end test case which uncovered this now passes.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #20181 from JoshRosen/SPARK-22984-fix-generate-unsaferow-joiner-bitmap-bugs.
2018-01-09 11:49:10 +08:00
Wenchen Fan eb45b52e82 [SPARK-21865][SQL] simplify the distribution semantic of Spark SQL
## What changes were proposed in this pull request?

**The current shuffle planning logic**

1. Each operator specifies the distribution requirements for its children, via the `Distribution` interface.
2. Each operator specifies its output partitioning, via the `Partitioning` interface.
3. `Partitioning.satisfy` determines whether a `Partitioning` can satisfy a `Distribution`.
4. For each operator, check each child of it, add a shuffle node above the child if the child partitioning can not satisfy the required distribution.
5. For each operator, check if its children's output partitionings are compatible with each other, via the `Partitioning.compatibleWith`.
6. If the check in 5 failed, add a shuffle above each child.
7. try to eliminate the shuffles added in 6, via `Partitioning.guarantees`.

This design has a major problem with the definition of "compatible".

`Partitioning.compatibleWith` is not well defined, ideally a `Partitioning` can't know if it's compatible with other `Partitioning`, without more information from the operator. For example, `t1 join t2 on t1.a = t2.b`, `HashPartitioning(a, 10)` should be compatible with `HashPartitioning(b, 10)` under this case, but the partitioning itself doesn't know it.

As a result, currently `Partitioning.compatibleWith` always return false except for literals, which make it almost useless. This also means, if an operator has distribution requirements for multiple children, Spark always add shuffle nodes to all the children(although some of them can be eliminated). However, there is no guarantee that the children's output partitionings are compatible with each other after adding these shuffles, we just assume that the operator will only specify `ClusteredDistribution` for multiple children.

I think it's very hard to guarantee children co-partition for all kinds of operators, and we can not even give a clear definition about co-partition between distributions like `ClusteredDistribution(a,b)` and `ClusteredDistribution(c)`.

I think we should drop the "compatible" concept in the distribution model, and let the operator achieve the co-partition requirement by special distribution requirements.

**Proposed shuffle planning logic after this PR**
(The first 4 are same as before)
1. Each operator specifies the distribution requirements for its children, via the `Distribution` interface.
2. Each operator specifies its output partitioning, via the `Partitioning` interface.
3. `Partitioning.satisfy` determines whether a `Partitioning` can satisfy a `Distribution`.
4. For each operator, check each child of it, add a shuffle node above the child if the child partitioning can not satisfy the required distribution.
5. For each operator, check if its children's output partitionings have the same number of partitions.
6. If the check in 5 failed, pick the max number of partitions from children's output partitionings, and add shuffle to child whose number of partitions doesn't equal to the max one.

The new distribution model is very simple, we only have one kind of relationship, which is `Partitioning.satisfy`. For multiple children, Spark only guarantees they have the same number of partitions, and it's the operator's responsibility to leverage this guarantee to achieve more complicated requirements. For example, non-broadcast joins can use the newly added `HashPartitionedDistribution` to achieve co-partition.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19080 from cloud-fan/exchange.
2018-01-08 19:41:41 +08:00
Josh Rosen 2c73d2a948 [SPARK-22983] Don't push filters beneath aggregates with empty grouping expressions
## What changes were proposed in this pull request?

The following SQL query should return zero rows, but in Spark it actually returns one row:

```
SELECT 1 from (
  SELECT 1 AS z,
  MIN(a.x)
  FROM (select 1 as x) a
  WHERE false
) b
where b.z != b.z
```

The problem stems from the `PushDownPredicate` rule: when this rule encounters a filter on top of an Aggregate operator, e.g. `Filter(Agg(...))`, it removes the original filter and adds a new filter onto Aggregate's child, e.g. `Agg(Filter(...))`. This is sometimes okay, but the case above is a counterexample: because there is no explicit `GROUP BY`, we are implicitly computing a global aggregate over the entire table so the original filter was not acting like a `HAVING` clause filtering the number of groups: if we push this filter then it fails to actually reduce the cardinality of the Aggregate output, leading to the wrong answer.

In 2016 I fixed a similar problem involving invalid pushdowns of data-independent filters (filters which reference no columns of the filtered relation). There was additional discussion after my fix was merged which pointed out that my patch was an incomplete fix (see #15289), but it looks I must have either misunderstood the comment or forgot to follow up on the additional points raised there.

This patch fixes the problem by choosing to never push down filters in cases where there are no grouping expressions. Since there are no grouping keys, the only columns are aggregate columns and we can't push filters defined over aggregate results, so this change won't cause us to miss out on any legitimate pushdown opportunities.

## How was this patch tested?

New regression tests in `SQLQueryTestSuite` and `FilterPushdownSuite`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #20180 from JoshRosen/SPARK-22983-dont-push-filters-beneath-aggs-with-empty-grouping-expressions.
2018-01-08 16:04:03 +08:00
Josh Rosen 71d65a3215 [SPARK-22985] Fix argument escaping bug in from_utc_timestamp / to_utc_timestamp codegen
## What changes were proposed in this pull request?

This patch adds additional escaping in `from_utc_timestamp` / `to_utc_timestamp` expression codegen in order to a bug where invalid timezones which contain special characters could cause generated code to fail to compile.

## How was this patch tested?

New regression tests in `DateExpressionsSuite`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #20182 from JoshRosen/SPARK-22985-fix-utc-timezone-function-escaping-bugs.
2018-01-08 11:39:45 +08:00
Takeshi Yamamuro 18e9414999 [SPARK-22973][SQL] Fix incorrect results of Casting Map to String
## What changes were proposed in this pull request?
This pr fixed the issue when casting maps into strings;
```
scala> Seq(Map(1 -> "a", 2 -> "b")).toDF("a").write.saveAsTable("t")
scala> sql("SELECT cast(a as String) FROM t").show(false)
+----------------------------------------------------------------+
|a                                                               |
+----------------------------------------------------------------+
|org.apache.spark.sql.catalyst.expressions.UnsafeMapData38bdd75d|
+----------------------------------------------------------------+
```
This pr modified the result into;
```
+----------------+
|a               |
+----------------+
|[1 -> a, 2 -> b]|
+----------------+
```

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20166 from maropu/SPARK-22973.
2018-01-07 13:42:01 +08:00
Takeshi Yamamuro e8af7e8aec [SPARK-22937][SQL] SQL elt output binary for binary inputs
## What changes were proposed in this pull request?
This pr modified `elt` to output binary for binary inputs.
`elt` in the current master always output data as a string. But, in some databases (e.g., MySQL), if all inputs are binary, `elt` also outputs binary (Also, this might be a small surprise).
This pr is related to #19977.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20135 from maropu/SPARK-22937.
2018-01-06 09:26:03 +08:00
Adrian Ionescu 51c33bd0d4 [SPARK-22961][REGRESSION] Constant columns should generate QueryPlanConstraints
## What changes were proposed in this pull request?

#19201 introduced the following regression: given something like `df.withColumn("c", lit(2))`, we're no longer picking up `c === 2` as a constraint and infer filters from it when joins are involved, which may lead to noticeable performance degradation.

This patch re-enables this optimization by picking up Aliases of Literals in Projection lists as constraints and making sure they're not treated as aliased columns.

## How was this patch tested?

Unit test was added.

Author: Adrian Ionescu <adrian@databricks.com>

Closes #20155 from adrian-ionescu/constant_constraints.
2018-01-05 21:32:39 +08:00
Takeshi Yamamuro 52fc5c17d9 [SPARK-22825][SQL] Fix incorrect results of Casting Array to String
## What changes were proposed in this pull request?
This pr fixed the issue when casting arrays into strings;
```
scala> val df = spark.range(10).select('id.cast("integer")).agg(collect_list('id).as('ids))
scala> df.write.saveAsTable("t")
scala> sql("SELECT cast(ids as String) FROM t").show(false)
+------------------------------------------------------------------+
|ids                                                               |
+------------------------------------------------------------------+
|org.apache.spark.sql.catalyst.expressions.UnsafeArrayData8bc285df|
+------------------------------------------------------------------+
```

This pr modified the result into;
```
+------------------------------+
|ids                           |
+------------------------------+
|[0, 1, 2, 3, 4, 5, 6, 7, 8, 9]|
+------------------------------+
```

## How was this patch tested?
Added tests in `CastSuite` and `SQLQuerySuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20024 from maropu/SPARK-22825.
2018-01-05 14:02:21 +08:00
Takeshi Yamamuro 6f68316e98 [SPARK-22771][SQL] Add a missing return statement in Concat.checkInputDataTypes
## What changes were proposed in this pull request?
This pr is a follow-up to fix a bug left in #19977.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20149 from maropu/SPARK-22771-FOLLOWUP.
2018-01-04 21:15:10 +08:00
Wenchen Fan 7d045c5f00 [SPARK-22944][SQL] improve FoldablePropagation
## What changes were proposed in this pull request?

`FoldablePropagation` is a little tricky as it needs to handle attributes that are miss-derived from children, e.g. outer join outputs. This rule does a kind of stop-able tree transform, to skip to apply this rule when hit a node which may have miss-derived attributes.

Logically we should be able to apply this rule above the unsupported nodes, by just treating the unsupported nodes as leaf nodes. This PR improves this rule to not stop the tree transformation, but reduce the foldable expressions that we want to propagate.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20139 from cloud-fan/foldable.
2018-01-04 13:14:52 +08:00
Sean Owen c284c4e1f6 [MINOR] Fix a bunch of typos 2018-01-02 07:10:19 +09:00
gatorsmile cfbe11e816 [SPARK-22895][SQL] Push down the deterministic predicates that are after the first non-deterministic
## What changes were proposed in this pull request?
Currently, we do not guarantee an order evaluation of conjuncts in either Filter or Join operator. This is also true to the mainstream RDBMS vendors like DB2 and MS SQL Server. Thus, we should also push down the deterministic predicates that are after the first non-deterministic, if possible.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20069 from gatorsmile/morePushDown.
2017-12-31 15:06:54 +08:00
Zhenhua Wang 234d9435d4 [TEST][MINOR] remove redundant EliminateSubqueryAliases in test code
## What changes were proposed in this pull request?

The `analyze` method in `implicit class DslLogicalPlan` already includes `EliminateSubqueryAliases`. So there's no need to call `EliminateSubqueryAliases` again after calling `analyze` in some test code.

## How was this patch tested?

Existing tests.

Author: Zhenhua Wang <wzh_zju@163.com>

Closes #20122 from wzhfy/redundant_code.
2017-12-30 20:48:39 +08:00
Takeshi Yamamuro f2b3525c17 [SPARK-22771][SQL] Concatenate binary inputs into a binary output
## What changes were proposed in this pull request?
This pr modified `concat` to concat binary inputs into a single binary output.
`concat` in the current master always output data as a string. But, in some databases (e.g., PostgreSQL), if all inputs are binary, `concat` also outputs binary.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #19977 from maropu/SPARK-22771.
2017-12-30 14:09:56 +08:00
oraviv fcf66a3276 [SPARK-21657][SQL] optimize explode quadratic memory consumpation
## What changes were proposed in this pull request?

The issue has been raised in two Jira tickets: [SPARK-21657](https://issues.apache.org/jira/browse/SPARK-21657), [SPARK-16998](https://issues.apache.org/jira/browse/SPARK-16998). Basically, what happens is that in collection generators like explode/inline we create many rows from each row. Currently each exploded row contains also the column on which it was created. This causes, for example, if we have a 10k array in one row that this array will get copy 10k times - to each of the row. this results a qudratic memory consumption. However, it is a common case that the original column gets projected out after the explode, so we can avoid duplicating it.
In this solution we propose to identify this situation in the optimizer and turn on a flag for omitting the original column in the generation process.

## How was this patch tested?

1. We added a benchmark test to MiscBenchmark that shows x16 improvement in runtimes.
2. We ran some of the other tests in MiscBenchmark and they show 15% improvements.
3. We ran this code on a specific case from our production data with rows containing arrays of size ~200k and it reduced the runtime from 6 hours to 3 mins.

Author: oraviv <oraviv@paypal.com>
Author: uzadude <ohad.raviv@gmail.com>
Author: uzadude <15645757+uzadude@users.noreply.github.com>

Closes #19683 from uzadude/optimize_explode.
2017-12-29 21:08:34 +08:00
Marco Gaido c6f01caded [SPARK-22750][SQL] Reuse mutable states when possible
## What changes were proposed in this pull request?

The PR introduces a new method `addImmutableStateIfNotExists ` to `CodeGenerator` to allow reusing and sharing the same global variable between different Expressions. This helps reducing the number of global variables needed, which is important to limit the impact on the constant pool.

## How was this patch tested?

added UTs

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

Closes #19940 from mgaido91/SPARK-22750.
2017-12-22 10:13:26 +08:00
Youngbin Kim 6e36d8d562 [SPARK-22829] Add new built-in function date_trunc()
## What changes were proposed in this pull request?

Adding date_trunc() as a built-in function.
`date_trunc` is common in other databases, but Spark or Hive does not have support for this. `date_trunc` is commonly used by data scientists and business intelligence application such as Superset (https://github.com/apache/incubator-superset).
We do have `trunc` but this only works with 'MONTH' and 'YEAR' level on the DateType input.

date_trunc() in other databases:
AWS Redshift: http://docs.aws.amazon.com/redshift/latest/dg/r_DATE_TRUNC.html
PostgreSQL: https://www.postgresql.org/docs/9.1/static/functions-datetime.html
Presto: https://prestodb.io/docs/current/functions/datetime.html

## How was this patch tested?

Unit tests

(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: Youngbin Kim <ykim828@hotmail.com>

Closes #20015 from youngbink/date_trunc.
2017-12-19 20:22:33 -08:00
Kazuaki Ishizaki ee56fc3432 [SPARK-18016][SQL] Code Generation: Constant Pool Limit - reduce entries for mutable state
## What changes were proposed in this pull request?

This PR is follow-on of #19518. This PR tries to reduce the number of constant pool entries used for accessing mutable state.
There are two directions:
1. Primitive type variables should be allocated at the outer class due to better performance. Otherwise, this PR allocates an array.
2. The length of allocated array is up to 32768 due to avoiding usage of constant pool entry at access (e.g. `mutableStateArray[32767]`).

Here are some discussions to determine these directions.
1. [[1]](https://github.com/apache/spark/pull/19518#issuecomment-346690464), [[2]](https://github.com/apache/spark/pull/19518#issuecomment-346690642), [[3]](https://github.com/apache/spark/pull/19518#issuecomment-346828180), [[4]](https://github.com/apache/spark/pull/19518#issuecomment-346831544), [[5]](https://github.com/apache/spark/pull/19518#issuecomment-346857340)
2. [[6]](https://github.com/apache/spark/pull/19518#issuecomment-346729172), [[7]](https://github.com/apache/spark/pull/19518#issuecomment-346798358), [[8]](https://github.com/apache/spark/pull/19518#issuecomment-346870408)

This PR modifies `addMutableState` function in the `CodeGenerator` to check if the declared state can be easily initialized compacted into an array. We identify three types of states that cannot compacted:

- Primitive type state (ints, booleans, etc) if the number of them does not exceed threshold
- Multiple-dimensional array type
- `inline = true`

When `useFreshName = false`, the given name is used.

Many codes were ported from #19518. Many efforts were put here. I think this PR should credit to bdrillard

With this PR, the following code is generated:
```
/* 005 */ class SpecificMutableProjection extends org.apache.spark.sql.catalyst.expressions.codegen.BaseMutableProjection {
/* 006 */
/* 007 */   private Object[] references;
/* 008 */   private InternalRow mutableRow;
/* 009 */   private boolean isNull_0;
/* 010 */   private boolean isNull_1;
/* 011 */   private boolean isNull_2;
/* 012 */   private int value_2;
/* 013 */   private boolean isNull_3;
...
/* 10006 */   private int value_4999;
/* 10007 */   private boolean isNull_5000;
/* 10008 */   private int value_5000;
/* 10009 */   private InternalRow[] mutableStateArray = new InternalRow[2];
/* 10010 */   private boolean[] mutableStateArray1 = new boolean[7001];
/* 10011 */   private int[] mutableStateArray2 = new int[1001];
/* 10012 */   private UTF8String[] mutableStateArray3 = new UTF8String[6000];
/* 10013 */
...
/* 107956 */     private void init_176() {
/* 107957 */       isNull_4986 = true;
/* 107958 */       value_4986 = -1;
...
/* 108004 */     }
...
```

## How was this patch tested?

Added a new test case to `GeneratedProjectionSuite`

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

Closes #19811 from kiszk/SPARK-18016.
2017-12-20 00:10:54 +08:00
Zhenhua Wang 571aa27554 [SPARK-21984][SQL] Join estimation based on equi-height histogram
## What changes were proposed in this pull request?

Equi-height histogram is one of the state-of-the-art statistics for cardinality estimation, which can provide better estimation accuracy, and good at cases with skew data.

This PR is to improve join estimation based on equi-height histogram. The difference from basic estimation (based on ndv) is the logic for computing join cardinality and the new ndv after join.

The main idea is as follows:
1. find overlapped ranges between two histograms from two join keys;
2. apply the formula `T(A IJ B) = T(A) * T(B) / max(V(A.k1), V(B.k1))` in each overlapped range.

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

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #19594 from wzhfy/join_estimation_histogram.
2017-12-19 21:55:21 +08:00
Wenchen Fan 2a29a60da3 Revert "[SPARK-22600][SQL] Fix 64kb limit for deeply nested expressions under wholestage codegen"
This reverts commit c7d0148615.
2017-12-14 11:22:23 +08:00
gatorsmile c5a4701acc Revert "[SPARK-21417][SQL] Infer join conditions using propagated constraints"
This reverts commit 6ac57fd0d1.
2017-12-13 11:50:04 -08:00
Wenchen Fan f6bcd3e53f [SPARK-22767][SQL] use ctx.addReferenceObj in InSet and ScalaUDF
## What changes were proposed in this pull request?

We should not operate on `references` directly in `Expression.doGenCode`, instead we should use the high-level API `addReferenceObj`.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19962 from cloud-fan/codegen.
2017-12-14 01:16:44 +08:00
gatorsmile 13e489b675 [SPARK-22759][SQL] Filters can be combined iff both are deterministic
## What changes were proposed in this pull request?
The query execution/optimization does not guarantee the expressions are evaluated in order. We only can combine them if and only if both are deterministic. We need to update the optimizer rule: CombineFilters.

## How was this patch tested?
Updated the existing tests.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19947 from gatorsmile/combineFilters.
2017-12-12 22:48:31 -08:00
Liang-Chi Hsieh c7d0148615 [SPARK-22600][SQL] Fix 64kb limit for deeply nested expressions under wholestage codegen
## What changes were proposed in this pull request?

SPARK-22543 fixes the 64kb compile error for deeply nested expression for non-wholestage codegen. This PR extends it to support wholestage codegen.

This patch brings some util methods in to extract necessary parameters for an expression if it is split to a function.

The util methods are put in object `ExpressionCodegen` under `codegen`. The main entry is `getExpressionInputParams` which returns all necessary parameters to evaluate the given expression in a split function.

This util methods can be used to split expressions too. This is a TODO item later.

## How was this patch tested?

Added test.

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

Closes #19813 from viirya/reduce-expr-code-for-wholestage.
2017-12-13 10:40:05 +08:00
Marco Gaido 4117786a87 [SPARK-22716][SQL] Avoid the creation of mutable states in addReferenceObj
## What changes were proposed in this pull request?

We have two methods to reference an object `addReferenceMinorObj` and `addReferenceObj `. The latter creates a new global variable, which means new entries in the constant pool.

The PR unifies the two method in a single `addReferenceObj` which returns the code to access the object in the `references` array and doesn't add new mutable states.

## How was this patch tested?

added UTs.

Author: Marco Gaido <mgaido@hortonworks.com>

Closes #19916 from mgaido91/SPARK-22716.
2017-12-13 10:29:14 +08:00
Ron Hu ecc179ecaa [SPARK-21322][SQL] support histogram in filter cardinality estimation
## What changes were proposed in this pull request?

Histogram is effective in dealing with skewed distribution. After we generate histogram information for column statistics, we need to adjust filter estimation based on histogram data structure.

## How was this patch tested?

We revised all the unit test cases by including histogram data structure.

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

Author: Ron Hu <ron.hu@huawei.com>

Closes #19783 from ron8hu/supportHistogram.
2017-12-12 15:04:49 +08:00
Marco Gaido b79071910e [SPARK-22696][SQL] objects functions should not use unneeded global variables
## What changes were proposed in this pull request?

Some objects functions are using global variables which are not needed. This can generate some unneeded entries in the constant pool.

The PR replaces the unneeded global variables with local variables.

## How was this patch tested?

added UTs

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

Closes #19908 from mgaido91/SPARK-22696.
2017-12-07 21:24:36 +08:00
Marco Gaido fc29446300 [SPARK-22699][SQL] GenerateSafeProjection should not use global variables for struct
## What changes were proposed in this pull request?

GenerateSafeProjection is defining a mutable state for each struct, which is not needed. This is bad for the well known issues related to constant pool limits.
The PR replace the global variable with a local one.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #19914 from mgaido91/SPARK-22699.
2017-12-07 21:18:27 +08:00
Kazuaki Ishizaki ea2fbf4197 [SPARK-22705][SQL] Case, Coalesce, and In use less global variables
## What changes were proposed in this pull request?

This PR accomplishes the following two items.

1. Reduce # of global variables from two to one for generated code of `Case` and `Coalesce` and remove global variables for generated code of `In`.
2. Make lifetime of global variable local within an operation

Item 1. reduces # of constant pool entries in a Java class. Item 2. ensures that an variable is not passed to arguments in a method split by `CodegenContext.splitExpressions()`, which is addressed by #19865.

## How was this patch tested?

Added new tests into `PredicateSuite`, `NullExpressionsSuite`, and `ConditionalExpressionSuite`.

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

Closes #19901 from kiszk/SPARK-22705.
2017-12-07 20:55:35 +08:00
Marco Gaido f110a7f884 [SPARK-22693][SQL] CreateNamedStruct and InSet should not use global variables
## What changes were proposed in this pull request?

CreateNamedStruct and InSet are using a global variable which is not needed. This can generate some unneeded entries in the constant pool.

The PR removes the unnecessary mutable states and makes them local variables.

## How was this patch tested?

added UT

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

Closes #19896 from mgaido91/SPARK-22693.
2017-12-06 14:12:16 -08:00
Marco Gaido e98f9647f4 [SPARK-22695][SQL] ScalaUDF should not use global variables
## What changes were proposed in this pull request?

ScalaUDF is using global variables which are not needed. This can generate some unneeded entries in the constant pool.

The PR replaces the unneeded global variables with local variables.

## How was this patch tested?

added UT

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

Closes #19900 from mgaido91/SPARK-22695.
2017-12-07 00:50:49 +08:00
Kazuaki Ishizaki 813c0f945d [SPARK-22704][SQL] Least and Greatest use less global variables
## What changes were proposed in this pull request?

This PR accomplishes the following two items.

1. Reduce # of global variables from two to one
2. Make lifetime of global variable local within an operation

Item 1. reduces # of constant pool entries in a Java class. Item 2. ensures that an variable is not passed to arguments in a method split by `CodegenContext.splitExpressions()`, which is addressed by #19865.

## How was this patch tested?

Added new test into `ArithmeticExpressionSuite`

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

Closes #19899 from kiszk/SPARK-22704.
2017-12-07 00:45:51 +08:00
Liang-Chi Hsieh 00d176d2fe [SPARK-20392][SQL] Set barrier to prevent re-entering a tree
## What changes were proposed in this pull request?

The SQL `Analyzer` goes through a whole query plan even most part of it is analyzed. This increases the time spent on query analysis for long pipelines in ML, especially.

This patch adds a logical node called `AnalysisBarrier` that wraps an analyzed logical plan to prevent it from analysis again. The barrier is applied to the analyzed logical plan in `Dataset`. It won't change the output of wrapped logical plan and just acts as a wrapper to hide it from analyzer. New operations on the dataset will be put on the barrier, so only the new nodes created will be analyzed.

This analysis barrier will be removed at the end of analysis stage.

## How was this patch tested?

Added tests.

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

Closes #19873 from viirya/SPARK-20392-reopen.
2017-12-05 21:43:41 -08:00
Zhenhua Wang 1e17ab83de [SPARK-22662][SQL] Failed to prune columns after rewriting predicate subquery
## What changes were proposed in this pull request?

As a simple example:
```
spark-sql> create table base (a int, b int) using parquet;
Time taken: 0.066 seconds
spark-sql> create table relInSubq ( x int, y int, z int) using parquet;
Time taken: 0.042 seconds
spark-sql> explain select a from base where a in (select x from relInSubq);
== Physical Plan ==
*Project [a#83]
+- *BroadcastHashJoin [a#83], [x#85], LeftSemi, BuildRight
   :- *FileScan parquet default.base[a#83,b#84] Batched: true, Format: Parquet, Location: InMemoryFileIndex[hdfs://100.0.0.4:9000/wzh/base], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<a:int,b:int>
   +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)))
      +- *Project [x#85]
         +- *FileScan parquet default.relinsubq[x#85] Batched: true, Format: Parquet, Location: InMemoryFileIndex[hdfs://100.0.0.4:9000/wzh/relinsubq], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<x:int>
```
We only need column `a` in table `base`, but all columns (`a`, `b`) are fetched.

The reason is that, in "Operator Optimizations" batch, `ColumnPruning` first produces a `Project` on table `base`, but then it's removed by `removeProjectBeforeFilter`. Because at that time, the predicate subquery is in filter form. Then, in "Rewrite Subquery" batch, `RewritePredicateSubquery` converts the subquery into a LeftSemi join, but this batch doesn't have the `ColumnPruning` rule. This results in reading all columns for the `base` table.

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

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #19855 from wzhfy/column_pruning_subquery.
2017-12-05 15:15:32 -08:00
Wenchen Fan a8af4da12c [SPARK-22682][SQL] HashExpression does not need to create global variables
## What changes were proposed in this pull request?

It turns out that `HashExpression` can pass around some values via parameter when splitting codes into methods, to save some global variable slots.

This can also prevent a weird case that global variable appears in parameter list, which is discovered by https://github.com/apache/spark/pull/19865

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19878 from cloud-fan/minor.
2017-12-05 12:43:05 +08:00
Marco Gaido 3887b7eef7 [SPARK-22665][SQL] Avoid repartitioning with empty list of expressions
## What changes were proposed in this pull request?

Repartitioning by empty set of expressions is currently possible, even though it is a case which is not handled properly. Indeed, in `HashExpression` there is a check to avoid to run it on an empty set, but this check is not performed while repartitioning.
Thus, the PR adds a check to avoid this wrong situation.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #19870 from mgaido91/SPARK-22665.
2017-12-04 17:08:56 -08:00
Marco Gaido 3927bb9b46 [SPARK-22473][FOLLOWUP][TEST] Remove deprecated Date functions
## What changes were proposed in this pull request?

#19696 replaced the deprecated usages for `Date` and `Waiter`, but a few methods were missed. The PR fixes the forgotten deprecated usages.

## How was this patch tested?

existing UTs

Author: Marco Gaido <mgaido@hortonworks.com>

Closes #19875 from mgaido91/SPARK-22473_FOLLOWUP.
2017-12-04 11:07:27 -06:00
Adrian Ionescu f5f8e84d9d [SPARK-22614] Dataset API: repartitionByRange(...)
## What changes were proposed in this pull request?

This PR introduces a way to explicitly range-partition a Dataset. So far, only round-robin and hash partitioning were possible via `df.repartition(...)`, but sometimes range partitioning might be desirable: e.g. when writing to disk, for better compression without the cost of global sort.

The current implementation piggybacks on the existing `RepartitionByExpression` `LogicalPlan` and simply adds the following logic: If its expressions are of type `SortOrder`, then it will do `RangePartitioning`; otherwise `HashPartitioning`. This was by far the least intrusive solution I could come up with.

## How was this patch tested?
Unit test for `RepartitionByExpression` changes, a test to ensure we're not changing the behavior of existing `.repartition()` and a few end-to-end tests in `DataFrameSuite`.

Author: Adrian Ionescu <adrian@databricks.com>

Closes #19828 from adrian-ionescu/repartitionByRange.
2017-11-30 15:41:34 -08:00
aokolnychyi 6ac57fd0d1 [SPARK-21417][SQL] Infer join conditions using propagated constraints
## What changes were proposed in this pull request?

This PR adds an optimization rule that infers join conditions using propagated constraints.

For instance, if there is a join, where the left relation has 'a = 1' and the right relation has 'b = 1', then the rule infers 'a = b' as a join predicate. Only semantically new predicates are appended to the existing join condition.

Refer to the corresponding ticket and tests for more details.

## How was this patch tested?

This patch comes with a new test suite to cover the implemented logic.

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

Closes #18692 from aokolnychyi/spark-21417.
2017-11-30 14:25:10 -08:00
Kazuaki Ishizaki 999ec137a9 [SPARK-22570][SQL] Avoid to create a lot of global variables by using a local variable with allocation of an object in generated code
## What changes were proposed in this pull request?

This PR reduces # of global variables in generated code by replacing a global variable with a local variable with an allocation of an object every time. When a lot of global variables were generated, the generated code may meet 64K constant pool limit.
This PR reduces # of generated global variables in the following three operations:
* `Cast` with String to primitive byte/short/int/long
* `RegExpReplace`
* `CreateArray`

I intentionally leave [this part](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala#L595-L603). This is because this variable keeps a class that is dynamically generated. In other word, it is not possible to reuse one class.

## How was this patch tested?

Added test cases

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

Closes #19797 from kiszk/SPARK-22570.
2017-12-01 02:28:24 +08:00
Wang Gengliang 57687280d4 [SPARK-22615][SQL] Handle more cases in PropagateEmptyRelation
## What changes were proposed in this pull request?

Currently, in the optimize rule `PropagateEmptyRelation`, the following cases is not handled:
1.  empty relation as right child in left outer join
2. empty relation as left child in right outer join
3. empty relation as right child  in left semi join
4. empty relation as right child  in left anti join
5. only one empty relation in full outer join

case 1 / 2 / 5 can be treated as **Cartesian product** and cause exception. See the new test cases.

## How was this patch tested?
Unit test

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #19825 from gengliangwang/SPARK-22615.
2017-11-29 09:17:39 -08:00
Marco Gaido 087879a77a [SPARK-22520][SQL] Support code generation for large CaseWhen
## What changes were proposed in this pull request?

Code generation is disabled for CaseWhen when the number of branches is higher than `spark.sql.codegen.maxCaseBranches` (which defaults to 20). This was done to prevent the well known 64KB method limit exception.
This PR proposes to support code generation also in those cases (without causing exceptions of course). As a side effect, we could get rid of the `spark.sql.codegen.maxCaseBranches` configuration.

## How was this patch tested?

existing UTs

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

Closes #19752 from mgaido91/SPARK-22520.
2017-11-28 07:46:18 +08:00
Kazuaki Ishizaki 2dbe275b2d [SPARK-22603][SQL] Fix 64KB JVM bytecode limit problem with FormatString
## What changes were proposed in this pull request?

This PR changes `FormatString` code generation to place generated code for expressions for arguments into separated methods if these size could be large.
This PR passes variable arguments by using an `Object` array.

## How was this patch tested?

Added new test cases into `StringExpressionSuite`

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

Closes #19817 from kiszk/SPARK-22603.
2017-11-27 20:32:01 +08:00
Kazuaki Ishizaki 554adc77d2 [SPARK-22595][SQL] fix flaky test: CastSuite.SPARK-22500: cast for struct should not generate codes beyond 64KB
## What changes were proposed in this pull request?

This PR reduces the number of fields in the test case of `CastSuite` to fix an issue that is pointed at [here](https://github.com/apache/spark/pull/19800#issuecomment-346634950).

```
java.lang.OutOfMemoryError: GC overhead limit exceeded
java.lang.OutOfMemoryError: GC overhead limit exceeded
	at org.codehaus.janino.UnitCompiler.findClass(UnitCompiler.java:10971)
	at org.codehaus.janino.UnitCompiler.findTypeByName(UnitCompiler.java:7607)
	at org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:5758)
	at org.codehaus.janino.UnitCompiler.getType2(UnitCompiler.java:5732)
	at org.codehaus.janino.UnitCompiler.access$13200(UnitCompiler.java:206)
	at org.codehaus.janino.UnitCompiler$18.visitReferenceType(UnitCompiler.java:5668)
	at org.codehaus.janino.UnitCompiler$18.visitReferenceType(UnitCompiler.java:5660)
	at org.codehaus.janino.Java$ReferenceType.accept(Java.java:3356)
	at org.codehaus.janino.UnitCompiler.getType(UnitCompiler.java:5660)
	at org.codehaus.janino.UnitCompiler.buildLocalVariableMap(UnitCompiler.java:2892)
	at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:2764)
	at org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:1262)
	at org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:1234)
	at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:538)
	at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:890)
	at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:894)
	at org.codehaus.janino.UnitCompiler.access$600(UnitCompiler.java:206)
	at org.codehaus.janino.UnitCompiler$2.visitMemberClassDeclaration(UnitCompiler.java:377)
	at org.codehaus.janino.UnitCompiler$2.visitMemberClassDeclaration(UnitCompiler.java:369)
	at org.codehaus.janino.Java$MemberClassDeclaration.accept(Java.java:1128)
	at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:369)
	at org.codehaus.janino.UnitCompiler.compileDeclaredMemberTypes(UnitCompiler.java:1209)
	at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:564)
	at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:890)
	at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:894)
	at org.codehaus.janino.UnitCompiler.access$600(UnitCompiler.java:206)
	at org.codehaus.janino.UnitCompiler$2.visitMemberClassDeclaration(UnitCompiler.java:377)
	at org.codehaus.janino.UnitCompiler$2.visitMemberClassDeclaration(UnitCompiler.java:369)
	at org.codehaus.janino.Java$MemberClassDeclaration.accept(Java.java:1128)
	at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:369)
	at org.codehaus.janino.UnitCompiler.compileDeclaredMemberTypes(UnitCompiler.java:1209)
	at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:564)
...
```

## How was this patch tested?

Used existing test case

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

Closes #19806 from kiszk/SPARK-22595.
2017-11-24 12:08:49 +01:00
Liang-Chi Hsieh 62a826f17c [SPARK-22591][SQL] GenerateOrdering shouldn't change CodegenContext.INPUT_ROW
## What changes were proposed in this pull request?

When I played with codegen in developing another PR, I found the value of `CodegenContext.INPUT_ROW` is not reliable. Under wholestage codegen, it is assigned to null first and then suddenly changed to `i`.

The reason is `GenerateOrdering` changes `CodegenContext.INPUT_ROW` but doesn't restore it back.

## How was this patch tested?

Added test.

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

Closes #19800 from viirya/SPARK-22591.
2017-11-24 11:46:58 +01:00
Wenchen Fan 0605ad7614 [SPARK-22543][SQL] fix java 64kb compile error for deeply nested expressions
## What changes were proposed in this pull request?

A frequently reported issue of Spark is the Java 64kb compile error. This is because Spark generates a very big method and it's usually caused by 3 reasons:

1. a deep expression tree, e.g. a very complex filter condition
2. many individual expressions, e.g. expressions can have many children, operators can have many expressions.
3. a deep query plan tree (with whole stage codegen)

This PR focuses on 1. There are already several patches(#15620  #18972 #18641) trying to fix this issue and some of them are already merged. However this is an endless job as every non-leaf expression has this issue.

This PR proposes to fix this issue in `Expression.genCode`, to make sure the code for a single expression won't grow too big.

According to maropu 's benchmark, no regression is found with TPCDS (thanks maropu !): https://docs.google.com/spreadsheets/d/1K3_7lX05-ZgxDXi9X_GleNnDjcnJIfoSlSCDZcL4gdg/edit?usp=sharing

## How was this patch tested?

existing test

Author: Wenchen Fan <wenchen@databricks.com>
Author: Wenchen Fan <cloud0fan@gmail.com>

Closes #19767 from cloud-fan/codegen.
2017-11-22 10:05:46 -08:00
Kazuaki Ishizaki ac10171bea [SPARK-22500][SQL] Fix 64KB JVM bytecode limit problem with cast
## What changes were proposed in this pull request?

This PR changes `cast` code generation to place generated code for expression for fields of a structure into separated methods if these size could be large.

## How was this patch tested?

Added new test cases into `CastSuite`

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

Closes #19730 from kiszk/SPARK-22500.
2017-11-21 22:24:43 +01:00
Kazuaki Ishizaki 9bdff0bcd8 [SPARK-22550][SQL] Fix 64KB JVM bytecode limit problem with elt
## What changes were proposed in this pull request?

This PR changes `elt` code generation to place generated code for expression for arguments into separated methods if these size could be large.
This PR resolved the case of `elt` with a lot of argument

## How was this patch tested?

Added new test cases into `StringExpressionsSuite`

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

Closes #19778 from kiszk/SPARK-22550.
2017-11-21 12:19:11 +01:00
Kazuaki Ishizaki c957714806 [SPARK-22508][SQL] Fix 64KB JVM bytecode limit problem with GenerateUnsafeRowJoiner.create()
## What changes were proposed in this pull request?

This PR changes `GenerateUnsafeRowJoiner.create()` code generation to place generated code for statements to operate bitmap and offset into separated methods if these size could be large.

## How was this patch tested?

Added a new test case into `GenerateUnsafeRowJoinerSuite`

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

Closes #19737 from kiszk/SPARK-22508.
2017-11-21 12:16:54 +01:00
Kazuaki Ishizaki 41c6f36018 [SPARK-22549][SQL] Fix 64KB JVM bytecode limit problem with concat_ws
## What changes were proposed in this pull request?

This PR changes `concat_ws` code generation to place generated code for expression for arguments into separated methods if these size could be large.
This PR resolved the case of `concat_ws` with a lot of argument

## How was this patch tested?

Added new test cases into `StringExpressionsSuite`

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

Closes #19777 from kiszk/SPARK-22549.
2017-11-21 01:42:05 +01:00
Kazuaki Ishizaki d54bfec2e0 [SPARK-22498][SQL] Fix 64KB JVM bytecode limit problem with concat
## What changes were proposed in this pull request?

This PR changes `concat` code generation to place generated code for expression for arguments into separated methods if these size could be large.
This PR resolved the case of `concat` with a lot of argument

## How was this patch tested?

Added new test cases into `StringExpressionsSuite`

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

Closes #19728 from kiszk/SPARK-22498.
2017-11-18 19:40:06 +01:00
Kazuaki Ishizaki 7f2e62ee6b [SPARK-22501][SQL] Fix 64KB JVM bytecode limit problem with in
## What changes were proposed in this pull request?

This PR changes `In` code generation to place generated code for expression for expressions for arguments into separated methods if these size could be large.

## How was this patch tested?

Added new test cases into `PredicateSuite`

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

Closes #19733 from kiszk/SPARK-22501.
2017-11-16 18:24:49 +01:00
Marco Gaido 4e7f07e255 [SPARK-22494][SQL] Fix 64KB limit exception with Coalesce and AtleastNNonNulls
## What changes were proposed in this pull request?

Both `Coalesce` and `AtLeastNNonNulls` can cause the 64KB limit exception when used with a lot of arguments and/or complex expressions.
This PR splits their expressions in order to avoid the issue.

## How was this patch tested?

Added UTs

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

Closes #19720 from mgaido91/SPARK-22494.
2017-11-16 18:19:13 +01:00
Kazuaki Ishizaki ed885e7a65 [SPARK-22499][SQL] Fix 64KB JVM bytecode limit problem with least and greatest
## What changes were proposed in this pull request?

This PR changes `least` and `greatest` code generation to place generated code for expression for arguments into separated methods if these size could be large.
This PR resolved two cases:

* `least` with a lot of argument
* `greatest` with a lot of argument

## How was this patch tested?

Added a new test case into `ArithmeticExpressionsSuite`

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

Closes #19729 from kiszk/SPARK-22499.
2017-11-16 17:56:21 +01:00
liutang123 bc0848b4c1 [SPARK-22469][SQL] Accuracy problem in comparison with string and numeric
## What changes were proposed in this pull request?
This fixes a problem caused by #15880
`select '1.5' > 0.5; // Result is NULL in Spark but is true in Hive.
`
When compare string and numeric, cast them as double like Hive.

Author: liutang123 <liutang123@yeah.net>

Closes #19692 from liutang123/SPARK-22469.
2017-11-15 09:02:54 -08:00
Kazuaki Ishizaki 9bf696dbec [SPARK-21720][SQL] Fix 64KB JVM bytecode limit problem with AND or OR
## What changes were proposed in this pull request?

This PR changes `AND` or `OR` code generation to place condition and then expressions' generated code into separated methods if these size could be large. When the method is newly generated, variables for `isNull` and `value` are declared as an instance variable to pass these values (e.g. `isNull1409` and `value1409`) to the callers of the generated method.

This PR resolved two cases:

* large code size of left expression
* large code size of right expression

## How was this patch tested?

Added a new test case into `CodeGenerationSuite`

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

Closes #18972 from kiszk/SPARK-21720.
2017-11-12 22:44:47 +01:00
Kazuaki Ishizaki f2da738c76 [SPARK-22284][SQL] Fix 64KB JVM bytecode limit problem in calculating hash for nested structs
## What changes were proposed in this pull request?

This PR avoids to generate a huge method for calculating a murmur3 hash for nested structs. This PR splits a huge method (e.g. `apply_4`) into multiple smaller methods.

Sample program
```
  val structOfString = new StructType().add("str", StringType)
  var inner = new StructType()
  for (_ <- 0 until 800) {
    inner = inner1.add("structOfString", structOfString)
  }
  var schema = new StructType()
  for (_ <- 0 until 50) {
    schema = schema.add("structOfStructOfStrings", inner)
  }
  GenerateMutableProjection.generate(Seq(Murmur3Hash(exprs, 42)))
```

Without this PR
```
/* 005 */ class SpecificMutableProjection extends org.apache.spark.sql.catalyst.expressions.codegen.BaseMutableProjection {
/* 006 */
/* 007 */   private Object[] references;
/* 008 */   private InternalRow mutableRow;
/* 009 */   private int value;
/* 010 */   private int value_0;
...
/* 034 */   public java.lang.Object apply(java.lang.Object _i) {
/* 035 */     InternalRow i = (InternalRow) _i;
/* 036 */
/* 037 */
/* 038 */
/* 039 */     value = 42;
/* 040 */     apply_0(i);
/* 041 */     apply_1(i);
/* 042 */     apply_2(i);
/* 043 */     apply_3(i);
/* 044 */     apply_4(i);
/* 045 */     nestedClassInstance.apply_5(i);
...
/* 089 */     nestedClassInstance8.apply_49(i);
/* 090 */     value_0 = value;
/* 091 */
/* 092 */     // copy all the results into MutableRow
/* 093 */     mutableRow.setInt(0, value_0);
/* 094 */     return mutableRow;
/* 095 */   }
/* 096 */
/* 097 */
/* 098 */   private void apply_4(InternalRow i) {
/* 099 */
/* 100 */     boolean isNull5 = i.isNullAt(4);
/* 101 */     InternalRow value5 = isNull5 ? null : (i.getStruct(4, 800));
/* 102 */     if (!isNull5) {
/* 103 */
/* 104 */       if (!value5.isNullAt(0)) {
/* 105 */
/* 106 */         final InternalRow element6400 = value5.getStruct(0, 1);
/* 107 */
/* 108 */         if (!element6400.isNullAt(0)) {
/* 109 */
/* 110 */           final UTF8String element6401 = element6400.getUTF8String(0);
/* 111 */           value = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(element6401.getBaseObject(), element6401.getBaseOffset(), element6401.numBytes(), value);
/* 112 */
/* 113 */         }
/* 114 */
/* 115 */
/* 116 */       }
/* 117 */
/* 118 */
/* 119 */       if (!value5.isNullAt(1)) {
/* 120 */
/* 121 */         final InternalRow element6402 = value5.getStruct(1, 1);
/* 122 */
/* 123 */         if (!element6402.isNullAt(0)) {
/* 124 */
/* 125 */           final UTF8String element6403 = element6402.getUTF8String(0);
/* 126 */           value = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(element6403.getBaseObject(), element6403.getBaseOffset(), element6403.numBytes(), value);
/* 127 */
/* 128 */         }
/* 128 */         }
/* 129 */
/* 130 */
/* 131 */       }
/* 132 */
/* 133 */
/* 134 */       if (!value5.isNullAt(2)) {
/* 135 */
/* 136 */         final InternalRow element6404 = value5.getStruct(2, 1);
/* 137 */
/* 138 */         if (!element6404.isNullAt(0)) {
/* 139 */
/* 140 */           final UTF8String element6405 = element6404.getUTF8String(0);
/* 141 */           value = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(element6405.getBaseObject(), element6405.getBaseOffset(), element6405.numBytes(), value);
/* 142 */
/* 143 */         }
/* 144 */
/* 145 */
/* 146 */       }
/* 147 */
...
/* 12074 */       if (!value5.isNullAt(798)) {
/* 12075 */
/* 12076 */         final InternalRow element7996 = value5.getStruct(798, 1);
/* 12077 */
/* 12078 */         if (!element7996.isNullAt(0)) {
/* 12079 */
/* 12080 */           final UTF8String element7997 = element7996.getUTF8String(0);
/* 12083 */         }
/* 12084 */
/* 12085 */
/* 12086 */       }
/* 12087 */
/* 12088 */
/* 12089 */       if (!value5.isNullAt(799)) {
/* 12090 */
/* 12091 */         final InternalRow element7998 = value5.getStruct(799, 1);
/* 12092 */
/* 12093 */         if (!element7998.isNullAt(0)) {
/* 12094 */
/* 12095 */           final UTF8String element7999 = element7998.getUTF8String(0);
/* 12096 */           value = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(element7999.getBaseObject(), element7999.getBaseOffset(), element7999.numBytes(), value);
/* 12097 */
/* 12098 */         }
/* 12099 */
/* 12100 */
/* 12101 */       }
/* 12102 */
/* 12103 */     }
/* 12104 */
/* 12105 */   }
/* 12106 */
/* 12106 */
/* 12107 */
/* 12108 */   private void apply_1(InternalRow i) {
...
```

With this PR
```
/* 005 */ class SpecificMutableProjection extends org.apache.spark.sql.catalyst.expressions.codegen.BaseMutableProjection {
/* 006 */
/* 007 */   private Object[] references;
/* 008 */   private InternalRow mutableRow;
/* 009 */   private int value;
/* 010 */   private int value_0;
/* 011 */
...
/* 034 */   public java.lang.Object apply(java.lang.Object _i) {
/* 035 */     InternalRow i = (InternalRow) _i;
/* 036 */
/* 037 */
/* 038 */
/* 039 */     value = 42;
/* 040 */     nestedClassInstance11.apply50_0(i);
/* 041 */     nestedClassInstance11.apply50_1(i);
...
/* 088 */     nestedClassInstance11.apply50_48(i);
/* 089 */     nestedClassInstance11.apply50_49(i);
/* 090 */     value_0 = value;
/* 091 */
/* 092 */     // copy all the results into MutableRow
/* 093 */     mutableRow.setInt(0, value_0);
/* 094 */     return mutableRow;
/* 095 */   }
/* 096 */
...
/* 37717 */   private void apply4_0(InternalRow value5, InternalRow i) {
/* 37718 */
/* 37719 */     if (!value5.isNullAt(0)) {
/* 37720 */
/* 37721 */       final InternalRow element6400 = value5.getStruct(0, 1);
/* 37722 */
/* 37723 */       if (!element6400.isNullAt(0)) {
/* 37724 */
/* 37725 */         final UTF8String element6401 = element6400.getUTF8String(0);
/* 37726 */         value = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(element6401.getBaseObject(), element6401.getBaseOffset(), element6401.numBytes(), value);
/* 37727 */
/* 37728 */       }
/* 37729 */
/* 37730 */
/* 37731 */     }
/* 37732 */
/* 37733 */     if (!value5.isNullAt(1)) {
/* 37734 */
/* 37735 */       final InternalRow element6402 = value5.getStruct(1, 1);
/* 37736 */
/* 37737 */       if (!element6402.isNullAt(0)) {
/* 37738 */
/* 37739 */         final UTF8String element6403 = element6402.getUTF8String(0);
/* 37740 */         value = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(element6403.getBaseObject(), element6403.getBaseOffset(), element6403.numBytes(), value);
/* 37741 */
/* 37742 */       }
/* 37743 */
/* 37744 */
/* 37745 */     }
/* 37746 */
/* 37747 */     if (!value5.isNullAt(2)) {
/* 37748 */
/* 37749 */       final InternalRow element6404 = value5.getStruct(2, 1);
/* 37750 */
/* 37751 */       if (!element6404.isNullAt(0)) {
/* 37752 */
/* 37753 */         final UTF8String element6405 = element6404.getUTF8String(0);
/* 37754 */         value = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(element6405.getBaseObject(), element6405.getBaseOffset(), element6405.numBytes(), value);
/* 37755 */
/* 37756 */       }
/* 37757 */
/* 37758 */
/* 37759 */     }
/* 37760 */
/* 37761 */   }
...
/* 218470 */
/* 218471 */     private void apply50_4(InternalRow i) {
/* 218472 */
/* 218473 */       boolean isNull5 = i.isNullAt(4);
/* 218474 */       InternalRow value5 = isNull5 ? null : (i.getStruct(4, 800));
/* 218475 */       if (!isNull5) {
/* 218476 */         apply4_0(value5, i);
/* 218477 */         apply4_1(value5, i);
/* 218478 */         apply4_2(value5, i);
...
/* 218742 */         nestedClassInstance.apply4_266(value5, i);
/* 218743 */       }
/* 218744 */
/* 218745 */     }
```

## How was this patch tested?

Added new test to `HashExpressionsSuite`

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

Closes #19563 from kiszk/SPARK-22284.
2017-11-10 21:17:49 +01:00
Shixiong Zhu 24ea781cd3 [SPARK-19644][SQL] Clean up Scala reflection garbage after creating Encoder
## What changes were proposed in this pull request?

Because of the memory leak issue in `scala.reflect.api.Types.TypeApi.<:<` (https://github.com/scala/bug/issues/8302), creating an encoder may leak memory.

This PR adds `cleanUpReflectionObjects` to clean up these leaking objects for methods calling `scala.reflect.api.Types.TypeApi.<:<`.

## How was this patch tested?

The updated unit tests.

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #19687 from zsxwing/SPARK-19644.
2017-11-10 11:27:28 -08:00
Wenchen Fan 0025ddeb1d [SPARK-22472][SQL] add null check for top-level primitive values
## What changes were proposed in this pull request?

One powerful feature of `Dataset` is, we can easily map SQL rows to Scala/Java objects and do runtime null check automatically.

For example, let's say we have a parquet file with schema `<a: int, b: string>`, and we have a `case class Data(a: Int, b: String)`. Users can easily read this parquet file into `Data` objects, and Spark will throw NPE if column `a` has null values.

However the null checking is left behind for top-level primitive values. For example, let's say we have a parquet file with schema `<a: Int>`, and we read it into Scala `Int`. If column `a` has null values, we will get some weird results.
```
scala> val ds = spark.read.parquet(...).as[Int]

scala> ds.show()
+----+
|v   |
+----+
|null|
|1   |
+----+

scala> ds.collect
res0: Array[Long] = Array(0, 1)

scala> ds.map(_ * 2).show
+-----+
|value|
+-----+
|-2   |
|2    |
+-----+
```

This is because internally Spark use some special default values for primitive types, but never expect users to see/operate these default value directly.

This PR adds null check for top-level primitive values

## How was this patch tested?

new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19707 from cloud-fan/bug.
2017-11-09 21:56:20 -08:00
Nathan Kronenfeld b57ed2245c [SPARK-22308][TEST-MAVEN] Support alternative unit testing styles in external applications
Continuation of PR#19528 (https://github.com/apache/spark/pull/19529#issuecomment-340252119)

The problem with the maven build in the previous PR was the new tests.... the creation of a spark session outside the tests meant there was more than one spark session around at a time.
I was using the spark session outside the tests so that the tests could share data; I've changed it so that each test creates the data anew.

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

Closes #19705 from nkronenfeld/alternative-style-tests-2.
2017-11-09 19:11:30 -08:00
jerryshao 6793a3dac0 [SPARK-22405][SQL] Add new alter table and alter database related ExternalCatalogEvent
## What changes were proposed in this pull request?

We're building a data lineage tool in which we need to monitor the metadata changes in ExternalCatalog, current ExternalCatalog already provides several useful events like "CreateDatabaseEvent" for custom SparkListener to use. But still there's some event missing, like alter database event and alter table event. So here propose to and new ExternalCatalogEvent.

## How was this patch tested?

Enrich the current UT and tested on local cluster.

CC hvanhovell please let me know your comments about current proposal, thanks.

Author: jerryshao <sshao@hortonworks.com>

Closes #19649 from jerryshao/SPARK-22405.
2017-11-09 11:57:56 +01:00
Liang-Chi Hsieh 40a8aefaf3 [SPARK-22442][SQL] ScalaReflection should produce correct field names for special characters
## What changes were proposed in this pull request?

For a class with field name of special characters, e.g.:
```scala
case class MyType(`field.1`: String, `field 2`: String)
```

Although we can manipulate DataFrame/Dataset, the field names are encoded:
```scala
scala> val df = Seq(MyType("a", "b"), MyType("c", "d")).toDF
df: org.apache.spark.sql.DataFrame = [field$u002E1: string, field$u00202: string]
scala> df.as[MyType].collect
res7: Array[MyType] = Array(MyType(a,b), MyType(c,d))
```

It causes resolving problem when we try to convert the data with non-encoded field names:
```scala
spark.read.json(path).as[MyType]
...
[info]   org.apache.spark.sql.AnalysisException: cannot resolve '`field$u002E1`' given input columns: [field 2, fie
ld.1];
[info]   at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
...
```

We should use decoded field name in Dataset schema.

## How was this patch tested?

Added tests.

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

Closes #19664 from viirya/SPARK-22442.
2017-11-09 11:54:50 +01:00
Dongjoon Hyun 98be55c0fa [SPARK-22222][CORE][TEST][FOLLOW-UP] Remove redundant and deprecated Timeouts
## What changes were proposed in this pull request?

Since SPARK-21939, Apache Spark uses `TimeLimits` instead of the deprecated `Timeouts`. This PR fixes the build warning `BufferHolderSparkSubmitSuite.scala` introduced at [SPARK-22222](https://github.com/apache/spark/pull/19460/files#diff-d8cf6e0c229969db94ec8ffc31a9239cR36) by removing the redundant `Timeouts`.
```scala
trait Timeouts in package concurrent is deprecated: Please use org.scalatest.concurrent.TimeLimits instead
[warn]     with Timeouts {
```
## How was this patch tested?

N/A

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19697 from dongjoon-hyun/SPARK-22222.
2017-11-09 16:34:38 +09:00
Kazuaki Ishizaki 3bba8621cf [SPARK-22378][SQL] Eliminate redundant null check in generated code for extracting an element from complex types
## What changes were proposed in this pull request?

This PR eliminates redundant null check in generated code for extracting an element from complex types `GetArrayItem`, `GetMapValue`, and `GetArrayStructFields`. Since these code generation does not take care of `nullable` in `DataType` such as `ArrayType`, the generated code always has `isNullAt(index)`.
This PR avoids to generate `isNullAt(index)` if `nullable` is false in `DataType`.

Example
```
    val nonNullArray = Literal.create(Seq(1), ArrayType(IntegerType, false))
    checkEvaluation(GetArrayItem(nonNullArray, Literal(0)), 1)
```

Before this PR
```
/* 034 */   public java.lang.Object apply(java.lang.Object _i) {
/* 035 */     InternalRow i = (InternalRow) _i;
/* 036 */
/* 037 */
/* 038 */
/* 039 */     boolean isNull = true;
/* 040 */     int value = -1;
/* 041 */
/* 042 */
/* 043 */
/* 044 */     isNull = false; // resultCode could change nullability.
/* 045 */
/* 046 */     final int index = (int) 0;
/* 047 */     if (index >= ((ArrayData) references[0]).numElements() || index < 0 || ((ArrayData) references[0]).isNullAt(index)) {
/* 048 */       isNull = true;
/* 049 */     } else {
/* 050 */       value = ((ArrayData) references[0]).getInt(index);
/* 051 */     }
/* 052 */     isNull_0 = isNull;
/* 053 */     value_0 = value;
/* 054 */
/* 055 */     // copy all the results into MutableRow
/* 056 */
/* 057 */     if (!isNull_0) {
/* 058 */       mutableRow.setInt(0, value_0);
/* 059 */     } else {
/* 060 */       mutableRow.setNullAt(0);
/* 061 */     }
/* 062 */
/* 063 */     return mutableRow;
/* 064 */   }
```

After this PR (Line 47 is changed)
```
/* 034 */   public java.lang.Object apply(java.lang.Object _i) {
/* 035 */     InternalRow i = (InternalRow) _i;
/* 036 */
/* 037 */
/* 038 */
/* 039 */     boolean isNull = true;
/* 040 */     int value = -1;
/* 041 */
/* 042 */
/* 043 */
/* 044 */     isNull = false; // resultCode could change nullability.
/* 045 */
/* 046 */     final int index = (int) 0;
/* 047 */     if (index >= ((ArrayData) references[0]).numElements() || index < 0) {
/* 048 */       isNull = true;
/* 049 */     } else {
/* 050 */       value = ((ArrayData) references[0]).getInt(index);
/* 051 */     }
/* 052 */     isNull_0 = isNull;
/* 053 */     value_0 = value;
/* 054 */
/* 055 */     // copy all the results into MutableRow
/* 056 */
/* 057 */     if (!isNull_0) {
/* 058 */       mutableRow.setInt(0, value_0);
/* 059 */     } else {
/* 060 */       mutableRow.setNullAt(0);
/* 061 */     }
/* 062 */
/* 063 */     return mutableRow;
/* 064 */   }
```

## How was this patch tested?

Added test cases into `ComplexTypeSuite`

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

Closes #19598 from kiszk/SPARK-22378.
2017-11-04 22:57:12 -07:00
Henry Robinson 6c6626614e [SPARK-22211][SQL] Remove incorrect FOJ limit pushdown
## What changes were proposed in this pull request?

It's not safe in all cases to push down a LIMIT below a FULL OUTER
JOIN. If the limit is pushed to one side of the FOJ, the physical
join operator can not tell if a row in the non-limited side would have a
match in the other side.

*If* the join operator guarantees that unmatched tuples from the limited
side are emitted before any unmatched tuples from the other side,
pushing down the limit is safe. But this is impractical for some join
implementations, e.g. SortMergeJoin.

For now, disable limit pushdown through a FULL OUTER JOIN, and we can
evaluate whether a more complicated solution is necessary in the future.

## How was this patch tested?

Ran org.apache.spark.sql.* tests. Altered full outer join tests in
LimitPushdownSuite.

Author: Henry Robinson <henry@cloudera.com>

Closes #19647 from henryr/spark-22211.
2017-11-04 22:47:25 -07:00
Wenchen Fan 2fd12af437 [SPARK-22306][SQL] alter table schema should not erase the bucketing metadata at hive side
forward-port https://github.com/apache/spark/pull/19622 to master branch.

This bug doesn't exist in master because we've added hive bucketing support and the hive bucketing metadata can be recognized by Spark, but we should still port it to master: 1) there may be other unsupported hive metadata removed by Spark. 2) reduce code difference between master and 2.2 to ease the backport in the feature.

***

When we alter table schema, we set the new schema to spark `CatalogTable`, convert it to hive table, and finally call `hive.alterTable`. This causes a problem in Spark 2.2, because hive bucketing metedata is not recognized by Spark, which means a Spark `CatalogTable` representing a hive table is always non-bucketed, and when we convert it to hive table and call `hive.alterTable`, the original hive bucketing metadata will be removed.

To fix this bug, we should read out the raw hive table metadata, update its schema, and call `hive.alterTable`. By doing this we can guarantee only the schema is changed, and nothing else.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19644 from cloud-fan/infer.
2017-11-02 23:41:16 +01:00
Henry Robinson 9f5c77ae32 [SPARK-21983][SQL] Fix Antlr 4.7 deprecation warnings
## What changes were proposed in this pull request?

Fix three deprecation warnings introduced by move to ANTLR 4.7:

* Use ParserRuleContext.addChild(TerminalNode) in preference to
  deprecated ParserRuleContext.addChild(Token) interface.
* TokenStream.reset() is deprecated in favour of seek(0)
* Replace use of deprecated ANTLRInputStream with stream returned by
  CharStreams.fromString()

The last item changed the way we construct ANTLR's input stream (from
direct instantiation to factory construction), so necessitated a change
to how we override the LA() method to always return an upper-case
char. The ANTLR object is now wrapped, rather than inherited-from.

* Also fix incorrect usage of CharStream.getText() which expects the rhs
  of the supplied interval to be the last char to be returned, i.e. the
  interval is inclusive, and work around bug in ANTLR 4.7 where empty
  streams or intervals may cause getText() to throw an error.

## How was this patch tested?

Ran all the sql tests. Confirmed that LA() override has coverage by
breaking it, and noting that tests failed.

Author: Henry Robinson <henry@apache.org>

Closes #19578 from henryr/spark-21983.
2017-10-30 07:45:54 +00:00
gatorsmile 659acf18da Revert "[SPARK-22308] Support alternative unit testing styles in external applications"
This reverts commit 592cfeab9c.
2017-10-29 10:37:25 -07:00
Wenchen Fan 7fdacbc77b [SPARK-19727][SQL][FOLLOWUP] Fix for round function that modifies original column
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/17075 , to fix the bug in codegen path.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19576 from cloud-fan/bug.
2017-10-28 18:24:18 -07:00
donnyzone c42d208e19 [SPARK-22333][SQL] timeFunctionCall(CURRENT_DATE, CURRENT_TIMESTAMP) has conflicts with columnReference
## What changes were proposed in this pull request?
https://issues.apache.org/jira/browse/SPARK-22333

In current version, users can use CURRENT_DATE() and CURRENT_TIMESTAMP() without specifying braces.
However, when a table has columns named as "current_date" or "current_timestamp", it will still be parsed as function call.

There are many such cases in our production cluster. We get the wrong answer due to this inappropriate behevior. In general, ColumnReference should get higher priority than timeFunctionCall.

## How was this patch tested?
unit test
manul test

Author: donnyzone <wellfengzhu@gmail.com>

Closes #19559 from DonnyZone/master.
2017-10-27 23:40:59 -07:00
Sathiya 01f6ba0e7a [SPARK-22181][SQL] Adds ReplaceExceptWithFilter rule
## What changes were proposed in this pull request?

Adds a new optimisation rule 'ReplaceExceptWithNotFilter' that replaces Except logical with Filter operator and schedule it before applying 'ReplaceExceptWithAntiJoin' rule. This way we can avoid expensive join operation if one or both of the datasets of the Except operation are fully derived out of Filters from a same parent.

## How was this patch tested?

The patch is tested locally using spark-shell + unit test.

Author: Sathiya <sathiya.kumar@polytechnique.edu>

Closes #19451 from sathiyapk/SPARK-22181-optimize-exceptWithFilter.
2017-10-27 18:57:08 -07:00
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