Commit graph

4566 commits

Author SHA1 Message Date
wm624@hotmail.com 22a9d064e9
[SPARK-14914][CORE] Fix Resource not closed after using, for unit tests and example
## What changes were proposed in this pull request?

This is a follow-up work of #15618.

Close file source;
For any newly created streaming context outside the withContext, explicitly close the context.

## How was this patch tested?

Existing unit tests.

Author: wm624@hotmail.com <wm624@hotmail.com>

Closes #15818 from wangmiao1981/rtest.
2016-11-10 10:54:36 +00:00
Wenchen Fan 6021c95a3a [SPARK-18147][SQL] do not fail for very complex aggregator result type
## What changes were proposed in this pull request?

~In `TypedAggregateExpression.evaluateExpression`, we may create `ReferenceToExpressions` with `CreateStruct`, and `CreateStruct` may generate too many codes and split them into several methods.  `ReferenceToExpressions` will replace `BoundReference` in `CreateStruct` with `LambdaVariable`, which can only be used as local variables and doesn't work if we split the generated code.~

It's already fixed by #15693 , this pr adds regression test

## How was this patch tested?

new test in `DatasetAggregatorSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15807 from cloud-fan/typed-agg.
2016-11-10 13:03:59 +08:00
Tyson Condie 3f62e1b5d9 [SPARK-17829][SQL] Stable format for offset log
## What changes were proposed in this pull request?

Currently we use java serialization for the WAL that stores the offsets contained in each batch. This has two main issues:
It can break across spark releases (though this is not the only thing preventing us from upgrading a running query)
It is unnecessarily opaque to the user.
I'd propose we require offsets to provide a user readable serialization and use that instead. JSON is probably a good option.
## How was this patch tested?

Tests were added for KafkaSourceOffset in [KafkaSourceOffsetSuite](external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala) and for LongOffset in [OffsetSuite](sql/core/src/test/scala/org/apache/spark/sql/streaming/OffsetSuite.scala)

Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request.

zsxwing marmbrus

Author: Tyson Condie <tcondie@gmail.com>
Author: Tyson Condie <tcondie@clash.local>

Closes #15626 from tcondie/spark-8360.
2016-11-09 15:03:22 -08:00
Herman van Hovell d8b81f778a [SPARK-18370][SQL] Add table information to InsertIntoHadoopFsRelationCommand
## What changes were proposed in this pull request?
`InsertIntoHadoopFsRelationCommand` does not keep track if it inserts into a table and what table it inserts to. This can make debugging these statements problematic. This PR adds table information the `InsertIntoHadoopFsRelationCommand`. Explaining this SQL command `insert into prq select * from range(0, 100000)` now yields the following executed plan:
```
== Physical Plan ==
ExecutedCommand
   +- InsertIntoHadoopFsRelationCommand file:/dev/assembly/spark-warehouse/prq, ParquetFormat, <function1>, Map(serialization.format -> 1, path -> file:/dev/assembly/spark-warehouse/prq), Append, CatalogTable(
	Table: `default`.`prq`
	Owner: hvanhovell
	Created: Wed Nov 09 17:42:30 CET 2016
	Last Access: Thu Jan 01 01:00:00 CET 1970
	Type: MANAGED
	Schema: [StructField(id,LongType,true)]
	Provider: parquet
	Properties: [transient_lastDdlTime=1478709750]
	Storage(Location: file:/dev/assembly/spark-warehouse/prq, InputFormat: org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat, OutputFormat: org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat, Serde: org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe, Properties: [serialization.format=1]))
         +- Project [id#7L]
            +- Range (0, 100000, step=1, splits=None)
```

## How was this patch tested?
Added extra checks to the `ParquetMetastoreSuite`

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

Closes #15832 from hvanhovell/SPARK-18370.
2016-11-09 12:26:09 -08:00
Ryan Blue d4028de976 [SPARK-18368][SQL] Fix regexp replace when serialized
## What changes were proposed in this pull request?

This makes the result value both transient and lazy, so that if the RegExpReplace object is initialized then serialized, `result: StringBuffer` will be correctly initialized.

## How was this patch tested?

* Verified that this patch fixed the query that found the bug.
* Added a test case that fails without the fix.

Author: Ryan Blue <blue@apache.org>

Closes #15834 from rdblue/SPARK-18368-fix-regexp-replace.
2016-11-09 11:00:53 -08:00
Yin Huai 47636618a5 Revert "[SPARK-18368] Fix regexp_replace with task serialization."
This reverts commit b9192bb3ff.
2016-11-09 10:47:29 -08:00
Cheng Lian 205e6d5867 [SPARK-18338][SQL][TEST-MAVEN] Fix test case initialization order under Maven builds
## What changes were proposed in this pull request?

Test case initialization order under Maven and SBT are different. Maven always creates instances of all test cases and then run them all together.

This fails `ObjectHashAggregateSuite` because the randomized test cases there register a temporary Hive function right before creating a test case, and can be cleared while initializing other successive test cases. In SBT, this is fine since the created test case is executed immediately after creating the temporary function.

To fix this issue, we should put initialization/destruction code into `beforeAll()` and `afterAll()`.

## How was this patch tested?

Existing tests.

Author: Cheng Lian <lian@databricks.com>

Closes #15802 from liancheng/fix-flaky-object-hash-agg-suite.
2016-11-09 09:49:02 -08:00
Dongjoon Hyun 02c5325b8f
[SPARK-18292][SQL] LogicalPlanToSQLSuite should not use resource dependent path for golden file generation
## What changes were proposed in this pull request?

`LogicalPlanToSQLSuite` uses the following command to update the existing answer files.

```bash
SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "hive/test-only *LogicalPlanToSQLSuite"
```

However, after introducing `getTestResourcePath`, it fails to update the previous golden answer files in the predefined directory. This issue aims to fix that.

## How was this patch tested?

It's a testsuite update. Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15789 from dongjoon-hyun/SPARK-18292.
2016-11-09 17:48:16 +00:00
gatorsmile e256392a12 [SPARK-17659][SQL] Partitioned View is Not Supported By SHOW CREATE TABLE
### What changes were proposed in this pull request?

`Partitioned View` is not supported by SPARK SQL. For Hive partitioned view, SHOW CREATE TABLE is unable to generate the right DDL. Thus, SHOW CREATE TABLE should not support it like the other Hive-only features. This PR is to issue an exception when detecting the view is a partitioned view.
### How was this patch tested?

Added a test case

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15233 from gatorsmile/partitionedView.
2016-11-09 00:11:48 -08:00
Ryan Blue b9192bb3ff [SPARK-18368] Fix regexp_replace with task serialization.
## What changes were proposed in this pull request?

This makes the result value both transient and lazy, so that if the RegExpReplace object is initialized then serialized, `result: StringBuffer` will be correctly initialized.

## How was this patch tested?

* Verified that this patch fixed the query that found the bug.
* Added a test case that fails without the fix.

Author: Ryan Blue <blue@apache.org>

Closes #15816 from rdblue/SPARK-18368-fix-regexp-replace.
2016-11-08 23:47:48 -08:00
Eric Liang 4afa39e223 [SPARK-18333][SQL] Revert hacks in parquet and orc reader to support case insensitive resolution
## What changes were proposed in this pull request?

These are no longer needed after https://issues.apache.org/jira/browse/SPARK-17183

cc cloud-fan

## How was this patch tested?

Existing parquet and orc tests.

Author: Eric Liang <ekl@databricks.com>

Closes #15799 from ericl/sc-4929.
2016-11-09 15:00:46 +08:00
Burak Yavuz 6f7ecb0f29 [SPARK-18342] Make rename failures fatal in HDFSBackedStateStore
## What changes were proposed in this pull request?

If the rename operation in the state store fails (`fs.rename` returns `false`), the StateStore should throw an exception and have the task retry. Currently if renames fail, nothing happens during execution immediately. However, you will observe that snapshot operations will fail, and then any attempt at recovery (executor failure / checkpoint recovery) also fails.

## How was this patch tested?

Unit test

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #15804 from brkyvz/rename-state.
2016-11-08 15:08:09 -08:00
jiangxingbo 9c419698fe [SPARK-18191][CORE] Port RDD API to use commit protocol
## What changes were proposed in this pull request?

This PR port RDD API to use commit protocol, the changes made here:
1. Add new internal helper class that saves an RDD using a Hadoop OutputFormat named `SparkNewHadoopWriter`, it's similar with `SparkHadoopWriter` but uses commit protocol. This class supports the newer `mapreduce` API, instead of the old `mapred` API which is supported by `SparkHadoopWriter`;
2. Rewrite `PairRDDFunctions.saveAsNewAPIHadoopDataset` function, so it uses commit protocol now.

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

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15769 from jiangxb1987/rdd-commit.
2016-11-08 09:41:01 -08:00
Wenchen Fan 73feaa30eb [SPARK-18346][SQL] TRUNCATE TABLE should fail if no partition is matched for the given non-partial partition spec
## What changes were proposed in this pull request?

a follow up of https://github.com/apache/spark/pull/15688

## How was this patch tested?

updated test in `DDLSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15805 from cloud-fan/truncate.
2016-11-08 22:28:29 +08:00
jiangxingbo 344dcad701 [SPARK-17868][SQL] Do not use bitmasks during parsing and analysis of CUBE/ROLLUP/GROUPING SETS
## What changes were proposed in this pull request?

We generate bitmasks for grouping sets during the parsing process, and use these during analysis. These bitmasks are difficult to work with in practice and have lead to numerous bugs. This PR removes these and use actual sets instead, however we still need to generate these offsets for the grouping_id.

This PR does the following works:
1. Replace bitmasks by actual grouping sets durning Parsing/Analysis stage of CUBE/ROLLUP/GROUPING SETS;
2. Add new testsuite `ResolveGroupingAnalyticsSuite` to test the `Analyzer.ResolveGroupingAnalytics` rule directly;
3. Fix a minor bug in `ResolveGroupingAnalytics`.
## How was this patch tested?

By existing test cases, and add new testsuite `ResolveGroupingAnalyticsSuite` to test directly.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15484 from jiangxb1987/group-set.
2016-11-08 15:11:03 +01:00
root c291bd2745 [SPARK-18137][SQL] Fix RewriteDistinctAggregates UnresolvedException when a UDAF has a foldable TypeCheck
## What changes were proposed in this pull request?

In RewriteDistinctAggregates rewrite funtion,after the UDAF's childs are mapped to AttributeRefference, If the UDAF(such as ApproximatePercentile) has a foldable TypeCheck for the input, It will failed because the AttributeRefference is not foldable,then the UDAF is not resolved, and then nullify on the unresolved object will throw a Exception.

In this PR, only map Unfoldable child to AttributeRefference, this can avoid the UDAF's foldable TypeCheck. and then only Expand Unfoldable child, there is no need to Expand a static value(foldable value).

**Before sql result**

> select percentile_approxy(key,0.99999),count(distinct key),sume(distinc key) from src limit 1
> org.apache.spark.sql.catalyst.analysis.UnresolvedException: Invalid call to dataType on unresolved object, tree: 'percentile_approx(CAST(src.`key` AS DOUBLE), CAST(0.99999BD AS DOUBLE), 10000)
> at org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute.dataType(unresolved.scala:92)
>     at org.apache.spark.sql.catalyst.optimizer.RewriteDistinctAggregates$.org$apache$spark$sql$catalyst$optimizer$RewriteDistinctAggregates$$nullify(RewriteDistinctAggregates.scala:261)

**After sql result**

> select percentile_approxy(key,0.99999),count(distinct key),sume(distinc key) from src limit 1
> [498.0,309,79136]
## How was this patch tested?

Add a test case in HiveUDFSuit.

Author: root <root@iZbp1gsnrlfzjxh82cz80vZ.(none)>

Closes #15668 from windpiger/RewriteDistinctUDAFUnresolveExcep.
2016-11-08 12:09:32 +01:00
Kazuaki Ishizaki 47731e1865 [SPARK-18207][SQL] Fix a compilation error due to HashExpression.doGenCode
## What changes were proposed in this pull request?

This PR avoids a compilation error due to more than 64KB Java byte code size. This error occur since  generate java code for computing a hash value for a row is too big. This PR fixes this compilation error by splitting a big code chunk into multiple methods by calling `CodegenContext.splitExpression` at `HashExpression.doGenCode`

The test case requires a calculation of hash code for a row that includes 1000 String fields. `HashExpression.doGenCode` generate a lot of Java code for this computation into one function. As a result, the size of the corresponding Java bytecode is more than 64 KB.

Generated code without this PR
````java
/* 027 */   public UnsafeRow apply(InternalRow i) {
/* 028 */     boolean isNull = false;
/* 029 */
/* 030 */     int value1 = 42;
/* 031 */
/* 032 */     boolean isNull2 = i.isNullAt(0);
/* 033 */     UTF8String value2 = isNull2 ? null : (i.getUTF8String(0));
/* 034 */     if (!isNull2) {
/* 035 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value2.getBaseObject(), value2.getBaseOffset(), value2.numBytes(), value1);
/* 036 */     }
/* 037 */
/* 038 */
/* 039 */     boolean isNull3 = i.isNullAt(1);
/* 040 */     UTF8String value3 = isNull3 ? null : (i.getUTF8String(1));
/* 041 */     if (!isNull3) {
/* 042 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value3.getBaseObject(), value3.getBaseOffset(), value3.numBytes(), value1);
/* 043 */     }
/* 044 */
/* 045 */
...
/* 7024 */
/* 7025 */     boolean isNull1001 = i.isNullAt(999);
/* 7026 */     UTF8String value1001 = isNull1001 ? null : (i.getUTF8String(999));
/* 7027 */     if (!isNull1001) {
/* 7028 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value1001.getBaseObject(), value1001.getBaseOffset(), value1001.numBytes(), value1);
/* 7029 */     }
/* 7030 */
/* 7031 */
/* 7032 */     boolean isNull1002 = i.isNullAt(1000);
/* 7033 */     UTF8String value1002 = isNull1002 ? null : (i.getUTF8String(1000));
/* 7034 */     if (!isNull1002) {
/* 7035 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value1002.getBaseObject(), value1002.getBaseOffset(), value1002.numBytes(), value1);
/* 7036 */     }
````

Generated code with this PR
````java
/* 3807 */   private void apply_249(InternalRow i) {
/* 3808 */
/* 3809 */     boolean isNull998 = i.isNullAt(996);
/* 3810 */     UTF8String value998 = isNull998 ? null : (i.getUTF8String(996));
/* 3811 */     if (!isNull998) {
/* 3812 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value998.getBaseObject(), value998.getBaseOffset(), value998.numBytes(), value1);
/* 3813 */     }
/* 3814 */
/* 3815 */     boolean isNull999 = i.isNullAt(997);
/* 3816 */     UTF8String value999 = isNull999 ? null : (i.getUTF8String(997));
/* 3817 */     if (!isNull999) {
/* 3818 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value999.getBaseObject(), value999.getBaseOffset(), value999.numBytes(), value1);
/* 3819 */     }
/* 3820 */
/* 3821 */     boolean isNull1000 = i.isNullAt(998);
/* 3822 */     UTF8String value1000 = isNull1000 ? null : (i.getUTF8String(998));
/* 3823 */     if (!isNull1000) {
/* 3824 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value1000.getBaseObject(), value1000.getBaseOffset(), value1000.numBytes(), value1);
/* 3825 */     }
/* 3826 */
/* 3827 */     boolean isNull1001 = i.isNullAt(999);
/* 3828 */     UTF8String value1001 = isNull1001 ? null : (i.getUTF8String(999));
/* 3829 */     if (!isNull1001) {
/* 3830 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value1001.getBaseObject(), value1001.getBaseOffset(), value1001.numBytes(), value1);
/* 3831 */     }
/* 3832 */
/* 3833 */   }
/* 3834 */
...
/* 4532 */   private void apply_0(InternalRow i) {
/* 4533 */
/* 4534 */     boolean isNull2 = i.isNullAt(0);
/* 4535 */     UTF8String value2 = isNull2 ? null : (i.getUTF8String(0));
/* 4536 */     if (!isNull2) {
/* 4537 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value2.getBaseObject(), value2.getBaseOffset(), value2.numBytes(), value1);
/* 4538 */     }
/* 4539 */
/* 4540 */     boolean isNull3 = i.isNullAt(1);
/* 4541 */     UTF8String value3 = isNull3 ? null : (i.getUTF8String(1));
/* 4542 */     if (!isNull3) {
/* 4543 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value3.getBaseObject(), value3.getBaseOffset(), value3.numBytes(), value1);
/* 4544 */     }
/* 4545 */
/* 4546 */     boolean isNull4 = i.isNullAt(2);
/* 4547 */     UTF8String value4 = isNull4 ? null : (i.getUTF8String(2));
/* 4548 */     if (!isNull4) {
/* 4549 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value4.getBaseObject(), value4.getBaseOffset(), value4.numBytes(), value1);
/* 4550 */     }
/* 4551 */
/* 4552 */     boolean isNull5 = i.isNullAt(3);
/* 4553 */     UTF8String value5 = isNull5 ? null : (i.getUTF8String(3));
/* 4554 */     if (!isNull5) {
/* 4555 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value5.getBaseObject(), value5.getBaseOffset(), value5.numBytes(), value1);
/* 4556 */     }
/* 4557 */
/* 4558 */   }
...
/* 7344 */   public UnsafeRow apply(InternalRow i) {
/* 7345 */     boolean isNull = false;
/* 7346 */
/* 7347 */     value1 = 42;
/* 7348 */     apply_0(i);
/* 7349 */     apply_1(i);
...
/* 7596 */     apply_248(i);
/* 7597 */     apply_249(i);
/* 7598 */     apply_250(i);
/* 7599 */     apply_251(i);
...
````

## How was this patch tested?

Add a new test in `DataFrameSuite`

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

Closes #15745 from kiszk/SPARK-18207.
2016-11-08 12:01:54 +01:00
gatorsmile 1da64e1fa0 [SPARK-18217][SQL] Disallow creating permanent views based on temporary views or UDFs
### What changes were proposed in this pull request?
Based on the discussion in [SPARK-18209](https://issues.apache.org/jira/browse/SPARK-18209). It doesn't really make sense to create permanent views based on temporary views or temporary UDFs.

To disallow the supports and issue the exceptions, this PR needs to detect whether a temporary view/UDF is being used when defining a permanent view. Basically, this PR can be split to two sub-tasks:

**Task 1:** detecting a temporary view from the query plan of view definition.
When finding an unresolved temporary view, Analyzer replaces it by a `SubqueryAlias` with the corresponding logical plan, which is stored in an in-memory HashMap. After replacement, it is impossible to detect whether the `SubqueryAlias` is added/generated from a temporary view. Thus, to detect the usage of a temporary view in view definition, this PR traverses the unresolved logical plan and uses the name of an `UnresolvedRelation` to detect whether it is a (global) temporary view.

**Task 2:** detecting a temporary UDF from the query plan of view definition.
Detecting usage of a temporary UDF in view definition is not straightfoward.

First, in the analyzed plan, we are having different forms to represent the functions. More importantly, some classes (e.g., `HiveGenericUDF`) are not accessible from `CreateViewCommand`, which is part of  `sql/core`. Thus, we used the unanalyzed plan `child` of `CreateViewCommand` to detect the usage of a temporary UDF. Because the plan has already been successfully analyzed, we can assume the functions have been defined/registered.

Second, in Spark, the functions have four forms: Spark built-in functions, built-in hash functions, permanent UDFs and temporary UDFs. We do not have any direct way to determine whether a function is temporary or not. Thus, we introduced a function `isTemporaryFunction` in `SessionCatalog`. This function contains the detailed logics to determine whether a function is temporary or not.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15764 from gatorsmile/blockTempFromPermViewCreation.
2016-11-07 18:34:21 -08:00
Liwei Lin c1a0c66bd2 [SPARK-18261][STRUCTURED STREAMING] Add statistics to MemorySink for joining
## What changes were proposed in this pull request?

Right now, there is no way to join the output of a memory sink with any table:

> UnsupportedOperationException: LeafNode MemoryPlan must implement statistics

This patch adds statistics to MemorySink, making joining snapshots of memory streams with tables possible.

## How was this patch tested?

Added a test case.

Author: Liwei Lin <lwlin7@gmail.com>

Closes #15786 from lw-lin/memory-sink-stat.
2016-11-07 17:49:24 -08:00
Ryan Blue 9b0593d5e9 [SPARK-18086] Add support for Hive session vars.
## What changes were proposed in this pull request?

This adds support for Hive variables:

* Makes values set via `spark-sql --hivevar name=value` accessible
* Adds `getHiveVar` and `setHiveVar` to the `HiveClient` interface
* Adds a SessionVariables trait for sessions like Hive that support variables (including Hive vars)
* Adds SessionVariables support to variable substitution
* Adds SessionVariables support to the SET command

## How was this patch tested?

* Adds a test to all supported Hive versions for accessing Hive variables
* Adds HiveVariableSubstitutionSuite

Author: Ryan Blue <blue@apache.org>

Closes #15738 from rdblue/SPARK-18086-add-hivevar-support.
2016-11-07 17:36:15 -08:00
hyukjinkwon 3eda05703f [SPARK-18295][SQL] Make to_json function null safe (matching it to from_json)
## What changes were proposed in this pull request?

This PR proposes to match up the behaviour of `to_json` to `from_json` function for null-safety.

Currently, it throws `NullPointException` but this PR fixes this to produce `null` instead.

with the data below:

```scala
import spark.implicits._

val df = Seq(Some(Tuple1(Tuple1(1))), None).toDF("a")
df.show()
```

```
+----+
|   a|
+----+
| [1]|
|null|
+----+
```

the codes below

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

df.select(to_json($"a")).show()
```

produces..

**Before**

throws `NullPointException` as below:

```
java.lang.NullPointerException
  at org.apache.spark.sql.catalyst.json.JacksonGenerator.org$apache$spark$sql$catalyst$json$JacksonGenerator$$writeFields(JacksonGenerator.scala:138)
  at org.apache.spark.sql.catalyst.json.JacksonGenerator$$anonfun$write$1.apply$mcV$sp(JacksonGenerator.scala:194)
  at org.apache.spark.sql.catalyst.json.JacksonGenerator.org$apache$spark$sql$catalyst$json$JacksonGenerator$$writeObject(JacksonGenerator.scala:131)
  at org.apache.spark.sql.catalyst.json.JacksonGenerator.write(JacksonGenerator.scala:193)
  at org.apache.spark.sql.catalyst.expressions.StructToJson.eval(jsonExpressions.scala:544)
  at org.apache.spark.sql.catalyst.expressions.Alias.eval(namedExpressions.scala:142)
  at org.apache.spark.sql.catalyst.expressions.InterpretedProjection.apply(Projection.scala:48)
  at org.apache.spark.sql.catalyst.expressions.InterpretedProjection.apply(Projection.scala:30)
  at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
```

**After**

```
+---------------+
|structtojson(a)|
+---------------+
|       {"_1":1}|
|           null|
+---------------+
```

## How was this patch tested?

Unit test in `JsonExpressionsSuite.scala` and `JsonFunctionsSuite.scala`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15792 from HyukjinKwon/SPARK-18295.
2016-11-07 16:54:40 -08:00
Josh Rosen 3a710b94b0 [SPARK-18236] Reduce duplicate objects in Spark UI and HistoryServer
## What changes were proposed in this pull request?

When profiling heap dumps from the HistoryServer and live Spark web UIs, I found a large amount of memory being wasted on duplicated objects and strings. This patch's changes remove most of this duplication, resulting in over 40% memory savings for some benchmarks.

- **Task metrics** (6441f0624dfcda9c7193a64bfb416a145b5aabdf): previously, every `TaskUIData` object would have its own instances of `InputMetricsUIData`, `OutputMetricsUIData`, `ShuffleReadMetrics`, and `ShuffleWriteMetrics`, but for many tasks these metrics are irrelevant because they're all zero. This patch changes how we construct these metrics in order to re-use a single immutable "empty" value for the cases where these metrics are empty.
- **TaskInfo.accumulables** (ade86db901127bf13c0e0bdc3f09c933a093bb76): Previously, every `TaskInfo` object had its own empty `ListBuffer` for holding updates from named accumulators. Tasks which didn't use named accumulators still paid for the cost of allocating and storing this empty buffer. To avoid this overhead, I changed the `val` with a mutable buffer into a `var` which holds an immutable Scala list, allowing tasks which do not have named accumulator updates to share the same singleton `Nil` object.
- **String.intern() in JSONProtocol** (7e05630e9a78c455db8c8c499f0590c864624e05): in the HistoryServer, executor hostnames and ids are deserialized from JSON, leading to massive duplication of these string objects. By calling `String.intern()` on the deserialized values we can remove all of this duplication. Since Spark now requires Java 7+ we don't have to worry about string interning exhausting the permgen (see http://java-performance.info/string-intern-in-java-6-7-8/).

## How was this patch tested?

I ran

```
sc.parallelize(1 to 100000, 100000).count()
```

in `spark-shell` with event logging enabled, then loaded that event log in the HistoryServer, performed a full GC, and took a heap dump. According to YourKit, the changes in this patch reduced memory consumption by roughly 28 megabytes (or 770k Java objects):

![image](https://cloud.githubusercontent.com/assets/50748/19953276/4f3a28aa-a129-11e6-93df-d7fa91396f66.png)

Here's a table illustrating the drop in objects due to deduplication (the drop is <100k for some objects because some events were dropped from the listener bus; this is a separate, existing bug that I'll address separately after CPU-profiling):

![image](https://cloud.githubusercontent.com/assets/50748/19953290/6a271290-a129-11e6-93ad-b825f1448886.png)

Author: Josh Rosen <joshrosen@databricks.com>

Closes #15743 from JoshRosen/spark-ui-memory-usage.
2016-11-07 16:14:19 -08:00
Kazuaki Ishizaki 19cf208063 [SPARK-17490][SQL] Optimize SerializeFromObject() for a primitive array
## What changes were proposed in this pull request?

Waiting for merging #13680

This PR optimizes `SerializeFromObject()` for an primitive array. This is derived from #13758 to address one of problems by using a simple way in #13758.

The current implementation always generates `GenericArrayData` from `SerializeFromObject()` for any type of an array in a logical plan. This involves a boxing at a constructor of `GenericArrayData` when `SerializedFromObject()` has an primitive array.

This PR enables to generate `UnsafeArrayData` from `SerializeFromObject()` for a primitive array. It can avoid boxing to create an instance of `ArrayData` in the generated code by Catalyst.

This PR also generate `UnsafeArrayData` in a case for `RowEncoder.serializeFor` or `CatalystTypeConverters.createToCatalystConverter`.

Performance improvement of `SerializeFromObject()` is up to 2.0x

```
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)

Without this PR
Write an array in Dataset:               Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            556 /  608         15.1          66.3       1.0X
Double                                        1668 / 1746          5.0         198.8       0.3X

with this PR
Write an array in Dataset:               Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            352 /  401         23.8          42.0       1.0X
Double                                         821 /  885         10.2          97.9       0.4X
```

Here is an example program that will happen in mllib as described in [SPARK-16070](https://issues.apache.org/jira/browse/SPARK-16070).

```
sparkContext.parallelize(Seq(Array(1, 2)), 1).toDS.map(e => e).show
```

Generated code before applying this PR

``` java
/* 039 */   protected void processNext() throws java.io.IOException {
/* 040 */     while (inputadapter_input.hasNext()) {
/* 041 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 042 */       int[] inputadapter_value = (int[])inputadapter_row.get(0, null);
/* 043 */
/* 044 */       Object mapelements_obj = ((Expression) references[0]).eval(null);
/* 045 */       scala.Function1 mapelements_value1 = (scala.Function1) mapelements_obj;
/* 046 */
/* 047 */       boolean mapelements_isNull = false || false;
/* 048 */       int[] mapelements_value = null;
/* 049 */       if (!mapelements_isNull) {
/* 050 */         Object mapelements_funcResult = null;
/* 051 */         mapelements_funcResult = mapelements_value1.apply(inputadapter_value);
/* 052 */         if (mapelements_funcResult == null) {
/* 053 */           mapelements_isNull = true;
/* 054 */         } else {
/* 055 */           mapelements_value = (int[]) mapelements_funcResult;
/* 056 */         }
/* 057 */
/* 058 */       }
/* 059 */       mapelements_isNull = mapelements_value == null;
/* 060 */
/* 061 */       serializefromobject_argIsNulls[0] = mapelements_isNull;
/* 062 */       serializefromobject_argValue = mapelements_value;
/* 063 */
/* 064 */       boolean serializefromobject_isNull = false;
/* 065 */       for (int idx = 0; idx < 1; idx++) {
/* 066 */         if (serializefromobject_argIsNulls[idx]) { serializefromobject_isNull = true; break; }
/* 067 */       }
/* 068 */
/* 069 */       final ArrayData serializefromobject_value = serializefromobject_isNull ? null : new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_argValue);
/* 070 */       serializefromobject_holder.reset();
/* 071 */
/* 072 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 073 */
/* 074 */       if (serializefromobject_isNull) {
/* 075 */         serializefromobject_rowWriter.setNullAt(0);
/* 076 */       } else {
/* 077 */         // Remember the current cursor so that we can calculate how many bytes are
/* 078 */         // written later.
/* 079 */         final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
/* 080 */
/* 081 */         if (serializefromobject_value instanceof UnsafeArrayData) {
/* 082 */           final int serializefromobject_sizeInBytes = ((UnsafeArrayData) serializefromobject_value).getSizeInBytes();
/* 083 */           // grow the global buffer before writing data.
/* 084 */           serializefromobject_holder.grow(serializefromobject_sizeInBytes);
/* 085 */           ((UnsafeArrayData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 086 */           serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
/* 087 */
/* 088 */         } else {
/* 089 */           final int serializefromobject_numElements = serializefromobject_value.numElements();
/* 090 */           serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 4);
/* 091 */
/* 092 */           for (int serializefromobject_index = 0; serializefromobject_index < serializefromobject_numElements; serializefromobject_index++) {
/* 093 */             if (serializefromobject_value.isNullAt(serializefromobject_index)) {
/* 094 */               serializefromobject_arrayWriter.setNullInt(serializefromobject_index);
/* 095 */             } else {
/* 096 */               final int serializefromobject_element = serializefromobject_value.getInt(serializefromobject_index);
/* 097 */               serializefromobject_arrayWriter.write(serializefromobject_index, serializefromobject_element);
/* 098 */             }
/* 099 */           }
/* 100 */         }
/* 101 */
/* 102 */         serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
/* 103 */       }
/* 104 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
/* 105 */       append(serializefromobject_result);
/* 106 */       if (shouldStop()) return;
/* 107 */     }
/* 108 */   }
/* 109 */ }
```

Generated code after applying this PR

``` java
/* 035 */   protected void processNext() throws java.io.IOException {
/* 036 */     while (inputadapter_input.hasNext()) {
/* 037 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 038 */       int[] inputadapter_value = (int[])inputadapter_row.get(0, null);
/* 039 */
/* 040 */       Object mapelements_obj = ((Expression) references[0]).eval(null);
/* 041 */       scala.Function1 mapelements_value1 = (scala.Function1) mapelements_obj;
/* 042 */
/* 043 */       boolean mapelements_isNull = false || false;
/* 044 */       int[] mapelements_value = null;
/* 045 */       if (!mapelements_isNull) {
/* 046 */         Object mapelements_funcResult = null;
/* 047 */         mapelements_funcResult = mapelements_value1.apply(inputadapter_value);
/* 048 */         if (mapelements_funcResult == null) {
/* 049 */           mapelements_isNull = true;
/* 050 */         } else {
/* 051 */           mapelements_value = (int[]) mapelements_funcResult;
/* 052 */         }
/* 053 */
/* 054 */       }
/* 055 */       mapelements_isNull = mapelements_value == null;
/* 056 */
/* 057 */       boolean serializefromobject_isNull = mapelements_isNull;
/* 058 */       final ArrayData serializefromobject_value = serializefromobject_isNull ? null : org.apache.spark.sql.catalyst.expressions.UnsafeArrayData.fromPrimitiveArray(mapelements_value);
/* 059 */       serializefromobject_isNull = serializefromobject_value == null;
/* 060 */       serializefromobject_holder.reset();
/* 061 */
/* 062 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 063 */
/* 064 */       if (serializefromobject_isNull) {
/* 065 */         serializefromobject_rowWriter.setNullAt(0);
/* 066 */       } else {
/* 067 */         // Remember the current cursor so that we can calculate how many bytes are
/* 068 */         // written later.
/* 069 */         final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
/* 070 */
/* 071 */         if (serializefromobject_value instanceof UnsafeArrayData) {
/* 072 */           final int serializefromobject_sizeInBytes = ((UnsafeArrayData) serializefromobject_value).getSizeInBytes();
/* 073 */           // grow the global buffer before writing data.
/* 074 */           serializefromobject_holder.grow(serializefromobject_sizeInBytes);
/* 075 */           ((UnsafeArrayData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 076 */           serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
/* 077 */
/* 078 */         } else {
/* 079 */           final int serializefromobject_numElements = serializefromobject_value.numElements();
/* 080 */           serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 4);
/* 081 */
/* 082 */           for (int serializefromobject_index = 0; serializefromobject_index < serializefromobject_numElements; serializefromobject_index++) {
/* 083 */             if (serializefromobject_value.isNullAt(serializefromobject_index)) {
/* 084 */               serializefromobject_arrayWriter.setNullInt(serializefromobject_index);
/* 085 */             } else {
/* 086 */               final int serializefromobject_element = serializefromobject_value.getInt(serializefromobject_index);
/* 087 */               serializefromobject_arrayWriter.write(serializefromobject_index, serializefromobject_element);
/* 088 */             }
/* 089 */           }
/* 090 */         }
/* 091 */
/* 092 */         serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
/* 093 */       }
/* 094 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
/* 095 */       append(serializefromobject_result);
/* 096 */       if (shouldStop()) return;
/* 097 */     }
/* 098 */   }
/* 099 */ }
```
## How was this patch tested?

Added a test in `DatasetSuite`, `RowEncoderSuite`, and `CatalystTypeConvertersSuite`

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

Closes #15044 from kiszk/SPARK-17490.
2016-11-08 00:14:57 +01:00
Weiqing Yang 0d95662e7f [SPARK-17108][SQL] Fix BIGINT and INT comparison failure in spark sql
## What changes were proposed in this pull request?

Add a function to check if two integers are compatible when invoking `acceptsType()` in `DataType`.
## How was this patch tested?

Manually.
E.g.

```
    spark.sql("create table t3(a map<bigint, array<string>>)")
    spark.sql("select * from t3 where a[1] is not null")
```

Before:

```
cannot resolve 't.`a`[1]' due to data type mismatch: argument 2 requires bigint type, however, '1' is of int type.; line 1 pos 22
org.apache.spark.sql.AnalysisException: cannot resolve 't.`a`[1]' due to data type mismatch: argument 2 requires bigint type, however, '1' is of int type.; line 1 pos 22
    at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
    at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1$$anonfun$apply$2.applyOrElse(CheckAnalysis.scala:82)
    at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1$$anonfun$apply$2.applyOrElse(CheckAnalysis.scala:74)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:307)
```

After:
 Run the sql queries above. No errors.

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #15448 from weiqingy/SPARK_17108.
2016-11-07 21:33:01 +01:00
Liang-Chi Hsieh a814eeac6b [SPARK-18125][SQL] Fix a compilation error in codegen due to splitExpression
## What changes were proposed in this pull request?

As reported in the jira, sometimes the generated java code in codegen will cause compilation error.

Code snippet to test it:

    case class Route(src: String, dest: String, cost: Int)
    case class GroupedRoutes(src: String, dest: String, routes: Seq[Route])

    val ds = sc.parallelize(Array(
      Route("a", "b", 1),
      Route("a", "b", 2),
      Route("a", "c", 2),
      Route("a", "d", 10),
      Route("b", "a", 1),
      Route("b", "a", 5),
      Route("b", "c", 6))
    ).toDF.as[Route]

    val grped = ds.map(r => GroupedRoutes(r.src, r.dest, Seq(r)))
      .groupByKey(r => (r.src, r.dest))
      .reduceGroups { (g1: GroupedRoutes, g2: GroupedRoutes) =>
        GroupedRoutes(g1.src, g1.dest, g1.routes ++ g2.routes)
      }.map(_._2)

The problem here is, in `ReferenceToExpressions` we evaluate the children vars to local variables. Then the result expression is evaluated to use those children variables. In the above case, the result expression code is too long and will be split by `CodegenContext.splitExpression`. So those local variables cannot be accessed and cause compilation error.

## How was this patch tested?

Jenkins tests.

Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request.

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

Closes #15693 from viirya/fix-codege-compilation-error.
2016-11-07 12:18:19 +01:00
gatorsmile 57626a5570 [SPARK-16904][SQL] Removal of Hive Built-in Hash Functions and TestHiveFunctionRegistry
### What changes were proposed in this pull request?

Currently, the Hive built-in `hash` function is not being used in Spark since Spark 2.0. The public interface does not allow users to unregister the Spark built-in functions. Thus, users will never use Hive's built-in `hash` function.

The only exception here is `TestHiveFunctionRegistry`, which allows users to unregister the built-in functions. Thus, we can load Hive's hash function in the test cases. If we disable it, 10+ test cases will fail because the results are different from the Hive golden answer files.

This PR is to remove `hash` from the list of `hiveFunctions` in `HiveSessionCatalog`. It will also remove `TestHiveFunctionRegistry`. This removal makes us easier to remove `TestHiveSessionState` in the future.
### How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14498 from gatorsmile/removeHash.
2016-11-07 01:16:37 -08:00
Reynold Xin 9db06c442c [SPARK-18296][SQL] Use consistent naming for expression test suites
## What changes were proposed in this pull request?
We have an undocumented naming convention to call expression unit tests ExpressionsSuite, and the end-to-end tests FunctionsSuite. It'd be great to make all test suites consistent with this naming convention.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #15793 from rxin/SPARK-18296.
2016-11-06 22:44:55 -08:00
Reynold Xin 07ac3f09da [SPARK-18167][SQL] Disable flaky hive partition pruning test. 2016-11-06 22:42:05 -08:00
Wenchen Fan 46b2e49993 [SPARK-18173][SQL] data source tables should support truncating partition
## What changes were proposed in this pull request?

Previously `TRUNCATE TABLE ... PARTITION` will always truncate the whole table for data source tables, this PR fixes it and improve `InMemoryCatalog` to make this command work with it.
## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15688 from cloud-fan/truncate.
2016-11-06 18:57:13 -08:00
hyukjinkwon 556a3b7d07 [SPARK-18269][SQL] CSV datasource should read null properly when schema is lager than parsed tokens
## What changes were proposed in this pull request?

Currently, there are the three cases when reading CSV by datasource when it is `PERMISSIVE` parse mode.

- schema == parsed tokens (from each line)
  No problem to cast the value in the tokens to the field in the schema as they are equal.

- schema < parsed tokens (from each line)
  It slices the tokens into the number of fields in schema.

- schema > parsed tokens (from each line)
  It appends `null` into parsed tokens so that safely values can be casted with the schema.

However, when `null` is appended in the third case, we should take `null` into account when casting the values.

In case of `StringType`, it is fine as `UTF8String.fromString(datum)` produces `null` when the input is `null`. Therefore, this case will happen only when schema is explicitly given and schema includes data types that are not `StringType`.

The codes below:

```scala
val path = "/tmp/a"
Seq("1").toDF().write.text(path.getAbsolutePath)
val schema = StructType(
  StructField("a", IntegerType, true) ::
  StructField("b", IntegerType, true) :: Nil)
spark.read.schema(schema).option("header", "false").csv(path).show()
```

prints

**Before**

```
java.lang.NumberFormatException: null
at java.lang.Integer.parseInt(Integer.java:542)
at java.lang.Integer.parseInt(Integer.java:615)
at scala.collection.immutable.StringLike$class.toInt(StringLike.scala:272)
at scala.collection.immutable.StringOps.toInt(StringOps.scala:29)
at org.apache.spark.sql.execution.datasources.csv.CSVTypeCast$.castTo(CSVInferSchema.scala:24)
```

**After**

```
+---+----+
|  a|   b|
+---+----+
|  1|null|
+---+----+
```

## How was this patch tested?

Unit test in `CSVSuite.scala` and `CSVTypeCastSuite.scala`

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15767 from HyukjinKwon/SPARK-18269.
2016-11-06 18:52:05 -08:00
hyukjinkwon 340f09d100
[SPARK-17854][SQL] rand/randn allows null/long as input seed
## What changes were proposed in this pull request?

This PR proposes `rand`/`randn` accept `null` as input in Scala/SQL and `LongType` as input in SQL. In this case, it treats the values as `0`.

So, this PR includes both changes below:
- `null` support

  It seems MySQL also accepts this.

  ``` sql
  mysql> select rand(0);
  +---------------------+
  | rand(0)             |
  +---------------------+
  | 0.15522042769493574 |
  +---------------------+
  1 row in set (0.00 sec)

  mysql> select rand(NULL);
  +---------------------+
  | rand(NULL)          |
  +---------------------+
  | 0.15522042769493574 |
  +---------------------+
  1 row in set (0.00 sec)
  ```

  and also Hive does according to [HIVE-14694](https://issues.apache.org/jira/browse/HIVE-14694)

  So the codes below:

  ``` scala
  spark.range(1).selectExpr("rand(null)").show()
  ```

  prints..

  **Before**

  ```
    Input argument to rand must be an integer literal.;; line 1 pos 0
  org.apache.spark.sql.AnalysisException: Input argument to rand must be an integer literal.;; line 1 pos 0
  at org.apache.spark.sql.catalyst.analysis.FunctionRegistry$$anonfun$5.apply(FunctionRegistry.scala:465)
  at org.apache.spark.sql.catalyst.analysis.FunctionRegistry$$anonfun$5.apply(FunctionRegistry.scala:444)
  ```

  **After**

  ```
    +-----------------------+
    |rand(CAST(NULL AS INT))|
    +-----------------------+
    |    0.13385709732307427|
    +-----------------------+
  ```
- `LongType` support in SQL.

  In addition, it make the function allows to take `LongType` consistently within Scala/SQL.

  In more details, the codes below:

  ``` scala
  spark.range(1).select(rand(1), rand(1L)).show()
  spark.range(1).selectExpr("rand(1)", "rand(1L)").show()
  ```

  prints..

  **Before**

  ```
  +------------------+------------------+
  |           rand(1)|           rand(1)|
  +------------------+------------------+
  |0.2630967864682161|0.2630967864682161|
  +------------------+------------------+

  Input argument to rand must be an integer literal.;; line 1 pos 0
  org.apache.spark.sql.AnalysisException: Input argument to rand must be an integer literal.;; line 1 pos 0
  at org.apache.spark.sql.catalyst.analysis.FunctionRegistry$$anonfun$5.apply(FunctionRegistry.scala:465)
  at
  ```

  **After**

  ```
  +------------------+------------------+
  |           rand(1)|           rand(1)|
  +------------------+------------------+
  |0.2630967864682161|0.2630967864682161|
  +------------------+------------------+

  +------------------+------------------+
  |           rand(1)|           rand(1)|
  +------------------+------------------+
  |0.2630967864682161|0.2630967864682161|
  +------------------+------------------+
  ```
## How was this patch tested?

Unit tests in `DataFrameSuite.scala` and `RandomSuite.scala`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15432 from HyukjinKwon/SPARK-17854.
2016-11-06 14:11:37 +00:00
hyukjinkwon 15d3926884 [MINOR][DOCUMENTATION] Fix some minor descriptions in functions consistently with expressions
## What changes were proposed in this pull request?

This PR proposes to improve documentation and fix some descriptions equivalent to several minor fixes identified in https://github.com/apache/spark/pull/15677

Also, this suggests to change `Note:` and `NOTE:` to `.. note::` consistently with the others which marks up pretty.

## How was this patch tested?

Jenkins tests and manually.

For PySpark, `Note:` and `NOTE:` to `.. note::` make the document as below:

**From**

![2016-11-04 6 53 35](https://cloud.githubusercontent.com/assets/6477701/20002648/42989922-a2c5-11e6-8a32-b73eda49e8c3.png)
![2016-11-04 6 53 45](https://cloud.githubusercontent.com/assets/6477701/20002650/429fb310-a2c5-11e6-926b-e030d7eb0185.png)
![2016-11-04 6 54 11](https://cloud.githubusercontent.com/assets/6477701/20002649/429d570a-a2c5-11e6-9e7e-44090f337e32.png)
![2016-11-04 6 53 51](https://cloud.githubusercontent.com/assets/6477701/20002647/4297fc74-a2c5-11e6-801a-b89fbcbfca44.png)
![2016-11-04 6 53 51](https://cloud.githubusercontent.com/assets/6477701/20002697/749f5780-a2c5-11e6-835f-022e1f2f82e3.png)

**To**

![2016-11-04 7 03 48](https://cloud.githubusercontent.com/assets/6477701/20002659/4961b504-a2c5-11e6-9ee0-ef0751482f47.png)
![2016-11-04 7 04 03](https://cloud.githubusercontent.com/assets/6477701/20002660/49871d3a-a2c5-11e6-85ea-d9a5d11efeff.png)
![2016-11-04 7 04 28](https://cloud.githubusercontent.com/assets/6477701/20002662/498e0f14-a2c5-11e6-803d-c0c5aeda4153.png)
![2016-11-04 7 33 39](https://cloud.githubusercontent.com/assets/6477701/20002731/a76e30d2-a2c5-11e6-993b-0481b8342d6b.png)
![2016-11-04 7 33 39](https://cloud.githubusercontent.com/assets/6477701/20002731/a76e30d2-a2c5-11e6-993b-0481b8342d6b.png)

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15765 from HyukjinKwon/minor-function-doc.
2016-11-05 21:47:33 -07:00
wangyang fb0d60814a [SPARK-17849][SQL] Fix NPE problem when using grouping sets
## What changes were proposed in this pull request?

Prior this pr, the following code would cause an NPE:
`case class point(a:String, b:String, c:String, d: Int)`

`val data = Seq(
point("1","2","3", 1),
point("4","5","6", 1),
point("7","8","9", 1)
)`
`sc.parallelize(data).toDF().registerTempTable("table")`
`spark.sql("select a, b, c, count(d) from table group by a, b, c GROUPING SETS ((a)) ").show()`

The reason is that when the grouping_id() behavior was changed in #10677, some code (which should be changed) was left out.

Take the above code for example, prior #10677, the bit mask for set "(a)" was `001`, while after #10677 the bit mask was changed to `011`. However, the `nonNullBitmask` was not changed accordingly.

This pr will fix this problem.
## How was this patch tested?

add integration tests

Author: wangyang <wangyang@haizhi.com>

Closes #15416 from yangw1234/groupingid.
2016-11-05 14:32:28 +01:00
hyukjinkwon a87471c830 [SPARK-18192][MINOR][FOLLOWUP] Missed json test in FileStreamSinkSuite
## What changes were proposed in this pull request?

This PR proposes to fix

```diff
 test("FileStreamSink - json") {
-  testFormat(Some("text"))
+  testFormat(Some("json"))
 }
```

`text` is being tested above

```
test("FileStreamSink - text") {
  testFormat(Some("text"))
}
```

## How was this patch tested?

Fixed test in `FileStreamSinkSuite.scala`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15785 from HyukjinKwon/SPARK-18192.
2016-11-05 13:41:35 +01:00
Reynold Xin e2648d3557 [SPARK-18287][SQL] Move hash expressions from misc.scala into hash.scala
## What changes were proposed in this pull request?
As the title suggests, this patch moves hash expressions from misc.scala into hash.scala, to make it easier to find the hash functions. I wanted to do this a while ago but decided to wait for the branch-2.1 cut so the chance of conflicts will be smaller.

## How was this patch tested?
Test cases were also moved out of MiscFunctionsSuite into HashExpressionsSuite.

Author: Reynold Xin <rxin@databricks.com>

Closes #15784 from rxin/SPARK-18287.
2016-11-05 11:29:17 +01:00
Wenchen Fan 95ec4e25bb [SPARK-17183][SPARK-17983][SPARK-18101][SQL] put hive serde table schema to table properties like data source table
## What changes were proposed in this pull request?

For data source tables, we will put its table schema, partition columns, etc. to table properties, to work around some hive metastore issues, e.g. not case-preserving, bad decimal type support, etc.

We should also do this for hive serde tables, to reduce the difference between hive serde tables and data source tables, e.g. column names should be case preserving.
## How was this patch tested?

existing tests, and a new test in `HiveExternalCatalog`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14750 from cloud-fan/minor1.
2016-11-05 00:58:50 -07:00
Burak Yavuz 6e27018157 [SPARK-18260] Make from_json null safe
## What changes were proposed in this pull request?

`from_json` is currently not safe against `null` rows. This PR adds a fix and a regression test for it.

## How was this patch tested?

Regression test

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #15771 from brkyvz/json_fix.
2016-11-05 00:07:51 -07:00
Reynold Xin 0f7c9e84e0 [SPARK-18189] [SQL] [Followup] Move test from ReplSuite to prevent java.lang.ClassCircularityError
closes #15774
2016-11-04 23:34:29 -07:00
Eric Liang 4cee2ce251 [SPARK-18167] Re-enable the non-flaky parts of SQLQuerySuite
## What changes were proposed in this pull request?

It seems the proximate cause of the test failures is that `cast(str as decimal)` in derby will raise an exception instead of returning NULL. This is a problem since Hive sometimes inserts `__HIVE_DEFAULT_PARTITION__` entries into the partition table as documented here: https://github.com/apache/hive/blob/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java#L1034

Basically, when these special default partitions are present, partition pruning pushdown using the SQL-direct mode will fail due this cast exception. As commented on in `MetaStoreDirectSql.java` above, this is normally fine since Hive falls back to JDO pruning, however when the pruning predicate contains an unsupported operator such as `>`, that will fail as well.

The only remaining question is why this behavior is nondeterministic. We know that when the test flakes, retries do not help, therefore the cause must be environmental. The current best hypothesis is that some config is different between different jenkins runs, which is why this PR prints out the Spark SQL and Hive confs for the test. The hope is that by comparing the config state for failure vs success we can isolate the root cause of the flakiness.

**Update:** we could not isolate the issue. It does not seem to be due to configuration differences. As such, I'm going to enable the non-flaky parts of the test since we are fairly confident these issues only occur with Derby (which is not used in production).

## How was this patch tested?

N/A

Author: Eric Liang <ekl@databricks.com>

Closes #15725 from ericl/print-confs-out.
2016-11-04 15:54:28 -07:00
Herman van Hovell 550cd56e8b [SPARK-17337][SQL] Do not pushdown predicates through filters with predicate subqueries
## What changes were proposed in this pull request?
The `PushDownPredicate` rule can create a wrong result if we try to push a filter containing a predicate subquery through a project when the subquery and the project share attributes (have the same source).

The current PR fixes this by making sure that we do not push down when there is a predicate subquery that outputs the same attributes as the filters new child plan.

## How was this patch tested?
Added a test to `SubquerySuite`. nsyca has done previous work this. I have taken test from his initial PR.

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

Closes #15761 from hvanhovell/SPARK-17337.
2016-11-04 21:18:13 +01:00
Herman van Hovell aa412c55e3 [SPARK-18259][SQL] Do not capture Throwable in QueryExecution
## What changes were proposed in this pull request?
`QueryExecution.toString` currently captures `java.lang.Throwable`s; this is far from a best practice and can lead to confusing situation or invalid application states. This PR fixes this by only capturing `AnalysisException`s.

## How was this patch tested?
Added a `QueryExecutionSuite`.

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

Closes #15760 from hvanhovell/SPARK-18259.
2016-11-03 21:59:59 -07:00
Reynold Xin f22954ad49 [SPARK-18257][SS] Improve error reporting for FileStressSuite
## What changes were proposed in this pull request?
This patch improves error reporting for FileStressSuite, when there is an error in Spark itself (not user code). This works by simply tightening the exception verification, and gets rid of the unnecessary thread for starting the stream.

Also renamed the class FileStreamStressSuite to make it more obvious it is a streaming suite.

## How was this patch tested?
This is a test only change and I manually verified error reporting by injecting some bug in the addBatch code for FileStreamSink.

Author: Reynold Xin <rxin@databricks.com>

Closes #15757 from rxin/SPARK-18257.
2016-11-03 15:30:45 -07:00
福星 16293311cd [SPARK-18237][HIVE] hive.exec.stagingdir have no effect
hive.exec.stagingdir have no effect in spark2.0.1,
Hive confs in hive-site.xml will be loaded in `hadoopConf`, so we should use `hadoopConf` in `InsertIntoHiveTable` instead of `SessionState.conf`

Author: 福星 <fuxing@wacai.com>

Closes #15744 from ClassNotFoundExp/master.
2016-11-03 12:02:01 -07:00
Reynold Xin b17057c0a6 [SPARK-18244][SQL] Rename partitionProviderIsHive -> tracksPartitionsInCatalog
## What changes were proposed in this pull request?
This patch renames partitionProviderIsHive to tracksPartitionsInCatalog, as the old name was too Hive specific.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #15750 from rxin/SPARK-18244.
2016-11-03 11:48:05 -07:00
Cheng Lian 27daf6bcde [SPARK-17949][SQL] A JVM object based aggregate operator
## What changes were proposed in this pull request?

This PR adds a new hash-based aggregate operator named `ObjectHashAggregateExec` that supports `TypedImperativeAggregate`, which may use arbitrary Java objects as aggregation states. Please refer to the [design doc](https://issues.apache.org/jira/secure/attachment/12834260/%5BDesign%20Doc%5D%20Support%20for%20Arbitrary%20Aggregation%20States.pdf) attached in [SPARK-17949](https://issues.apache.org/jira/browse/SPARK-17949) for more details about it.

The major benefit of this operator is better performance when evaluating `TypedImperativeAggregate` functions, especially when there are relatively few distinct groups. Functions like Hive UDAFs, `collect_list`, and `collect_set` may also benefit from this after being migrated to `TypedImperativeAggregate`.

The following feature flag is introduced to enable or disable the new aggregate operator:
- Name: `spark.sql.execution.useObjectHashAggregateExec`
- Default value: `true`

We can also configure the fallback threshold using the following SQL operation:
- Name: `spark.sql.objectHashAggregate.sortBased.fallbackThreshold`
- Default value: 128

  Fallback to sort-based aggregation when more than 128 distinct groups are accumulated in the aggregation hash map. This number is intentionally made small to avoid GC problems since aggregation buffers of this operator may contain arbitrary Java objects.

  This may be improved by implementing size tracking for this operator, but that can be done in a separate PR.

Code generation and size tracking are planned to be implemented in follow-up PRs.
## Benchmark results
### `ObjectHashAggregateExec` vs `SortAggregateExec`

The first benchmark compares `ObjectHashAggregateExec` and `SortAggregateExec` by evaluating `typed_count`, a testing `TypedImperativeAggregate` version of the SQL `count` function.

```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.5
Intel(R) Core(TM) i7-4960HQ CPU  2.60GHz

object agg v.s. sort agg:                Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
sort agg w/ group by                        31251 / 31908          3.4         298.0       1.0X
object agg w/ group by w/o fallback           6903 / 7141         15.2          65.8       4.5X
object agg w/ group by w/ fallback          20945 / 21613          5.0         199.7       1.5X
sort agg w/o group by                         4734 / 5463         22.1          45.2       6.6X
object agg w/o group by w/o fallback          4310 / 4529         24.3          41.1       7.3X
```

The next benchmark compares `ObjectHashAggregateExec` and `SortAggregateExec` by evaluating the Spark native version of `percentile_approx`.

Note that `percentile_approx` is so heavy an aggregate function that the bottleneck of the benchmark is evaluating the aggregate function itself rather than the aggregate operator since I couldn't run a large scale benchmark on my laptop. That's why the results are so close and looks counter-intuitive (aggregation with grouping is even faster than that aggregation without grouping).

```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.5
Intel(R) Core(TM) i7-4960HQ CPU  2.60GHz

object agg v.s. sort agg:                Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
sort agg w/ group by                          3418 / 3530          0.6        1630.0       1.0X
object agg w/ group by w/o fallback           3210 / 3314          0.7        1530.7       1.1X
object agg w/ group by w/ fallback            3419 / 3511          0.6        1630.1       1.0X
sort agg w/o group by                         4336 / 4499          0.5        2067.3       0.8X
object agg w/o group by w/o fallback          4271 / 4372          0.5        2036.7       0.8X
```
### Hive UDAF vs Spark AF

This benchmark compares the following two kinds of aggregate functions:
- "hive udaf": Hive implementation of `percentile_approx`, without partial aggregation supports, evaluated using `SortAggregateExec`.
- "spark af": Spark native implementation of `percentile_approx`, with partial aggregation support, evaluated using `ObjectHashAggregateExec`

The performance differences are mostly due to faster implementation and partial aggregation support in the Spark native version of `percentile_approx`.

This benchmark basically shows the performance differences between the worst case, where an aggregate function without partial aggregation support is evaluated using `SortAggregateExec`, and the best case, where a `TypedImperativeAggregate` with partial aggregation support is evaluated using `ObjectHashAggregateExec`.

```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.5
Intel(R) Core(TM) i7-4960HQ CPU  2.60GHz

hive udaf vs spark af:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
hive udaf w/o group by                        5326 / 5408          0.0       81264.2       1.0X
spark af w/o group by                           93 /  111          0.7        1415.6      57.4X
hive udaf w/ group by                         3804 / 3946          0.0       58050.1       1.4X
spark af w/ group by w/o fallback               71 /   90          0.9        1085.7      74.8X
spark af w/ group by w/ fallback                98 /  111          0.7        1501.6      54.1X
```
### Real world benchmark

We also did a relatively large benchmark using a real world query involving `percentile_approx`:
- Hive UDAF implementation, sort-based aggregation, w/o partial aggregation support

  24.77 minutes
- Native implementation, sort-based aggregation, w/ partial aggregation support

  4.64 minutes
- Native implementation, object hash aggregator, w/ partial aggregation support

  1.80 minutes
## How was this patch tested?

New unit tests and randomized test cases are added in `ObjectAggregateFunctionSuite`.

Author: Cheng Lian <lian@databricks.com>

Closes #15590 from liancheng/obj-hash-agg.
2016-11-03 09:34:51 -07:00
gatorsmile 66a99f4a41 [SPARK-17981][SPARK-17957][SQL] Fix Incorrect Nullability Setting to False in FilterExec
### What changes were proposed in this pull request?

When `FilterExec` contains `isNotNull`, which could be inferred and pushed down or users specified, we convert the nullability of the involved columns if the top-layer expression is null-intolerant. However, this is not correct, if the top-layer expression is not a leaf expression, it could still tolerate the null when it has null-tolerant child expressions.

For example, `cast(coalesce(a#5, a#15) as double)`. Although `cast` is a null-intolerant expression, but obviously`coalesce` is null-tolerant. Thus, it could eat null.

When the nullability is wrong, we could generate incorrect results in different cases. For example,

``` Scala
    val df1 = Seq((1, 2), (2, 3)).toDF("a", "b")
    val df2 = Seq((2, 5), (3, 4)).toDF("a", "c")
    val joinedDf = df1.join(df2, Seq("a"), "outer").na.fill(0)
    val df3 = Seq((3, 1)).toDF("a", "d")
    joinedDf.join(df3, "a").show
```

The optimized plan is like

```
Project [a#29, b#30, c#31, d#42]
+- Join Inner, (a#29 = a#41)
   :- Project [cast(coalesce(cast(coalesce(a#5, a#15) as double), 0.0) as int) AS a#29, cast(coalesce(cast(b#6 as double), 0.0) as int) AS b#30, cast(coalesce(cast(c#16 as double), 0.0) as int) AS c#31]
   :  +- Filter isnotnull(cast(coalesce(cast(coalesce(a#5, a#15) as double), 0.0) as int))
   :     +- Join FullOuter, (a#5 = a#15)
   :        :- LocalRelation [a#5, b#6]
   :        +- LocalRelation [a#15, c#16]
   +- LocalRelation [a#41, d#42]
```

Without the fix, it returns an empty result. With the fix, it can return a correct answer:

```
+---+---+---+---+
|  a|  b|  c|  d|
+---+---+---+---+
|  3|  0|  4|  1|
+---+---+---+---+
```
### How was this patch tested?

Added test cases to verify the nullability changes in FilterExec. Also added a test case for verifying the reported incorrect result.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15523 from gatorsmile/nullabilityFilterExec.
2016-11-03 16:35:36 +01:00
Reynold Xin 0ea5d5b24c [SQL] minor - internal doc improvement for InsertIntoTable.
## What changes were proposed in this pull request?
I was reading this part of the code and was really confused by the "partition" parameter. This patch adds some documentation for it to reduce confusion in the future.

I also looked around other logical plans but most of them are either already documented, or pretty self-evident to people that know Spark SQL.

## How was this patch tested?
N/A - doc change only.

Author: Reynold Xin <rxin@databricks.com>

Closes #15749 from rxin/doc-improvement.
2016-11-03 02:45:54 -07:00
Reynold Xin 937af592e6 [SPARK-18219] Move commit protocol API (internal) from sql/core to core module
## What changes were proposed in this pull request?
This patch moves the new commit protocol API from sql/core to core module, so we can use it in the future in the RDD API.

As part of this patch, I also moved the speficiation of the random uuid for the write path out of the commit protocol, and instead pass in a job id.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #15731 from rxin/SPARK-18219.
2016-11-03 02:42:48 -07:00
Daoyuan Wang 96cc1b5675 [SPARK-17122][SQL] support drop current database
## What changes were proposed in this pull request?

In Spark 1.6 and earlier, we can drop the database we are using. In Spark 2.0, native implementation prevent us from dropping current database, which may break some old queries. This PR would re-enable the feature.
## How was this patch tested?

one new unit test in `SessionCatalogSuite`.

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

Closes #15011 from adrian-wang/dropcurrent.
2016-11-03 00:18:03 -07:00
gatorsmile 9ddec8636c [SPARK-18175][SQL] Improve the test case coverage of implicit type casting
### What changes were proposed in this pull request?

So far, we have limited test case coverage about implicit type casting. We need to draw a matrix to find all the possible casting pairs.
- Reorged the existing test cases
- Added all the possible type casting pairs
- Drawed a matrix to show the implicit type casting. The table is very wide. Maybe hard to review. Thus, you also can access the same table via the link to [a google sheet](https://docs.google.com/spreadsheets/d/19PS4ikrs-Yye_mfu-rmIKYGnNe-NmOTt5DDT1fOD3pI/edit?usp=sharing).

SourceType\CastToType | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | BinaryType | BooleanType | StringType | DateType | TimestampType | ArrayType | MapType | StructType | NullType | CalendarIntervalType | DecimalType | NumericType | IntegralType
------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ |  -----------
**ByteType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X    | X    | StringType | X    | X    | X    | X    | X    | X    | X    | DecimalType(3, 0) | ByteType | ByteType
**ShortType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X    | X    | StringType | X    | X    | X    | X    | X    | X    | X    | DecimalType(5, 0) | ShortType | ShortType
**IntegerType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X    | X    | StringType | X    | X    | X    | X    | X    | X    | X    | DecimalType(10, 0) | IntegerType | IntegerType
**LongType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X    | X    | StringType | X    | X    | X    | X    | X    | X    | X    | DecimalType(20, 0) | LongType | LongType
**DoubleType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X    | X    | StringType | X    | X    | X    | X    | X    | X    | X    | DecimalType(30, 15) | DoubleType | IntegerType
**FloatType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X    | X    | StringType | X    | X    | X    | X    | X    | X    | X    | DecimalType(14, 7) | FloatType | IntegerType
**Dec(10, 2)** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X    | X    | StringType | X    | X    | X    | X    | X    | X    | X    | DecimalType(10, 2) | Dec(10, 2) | IntegerType
**BinaryType** | X    | X    | X    | X    | X    | X    | X    | BinaryType | X    | StringType | X    | X    | X    | X    | X    | X    | X    | X    | X    | X
**BooleanType** | X    | X    | X    | X    | X    | X    | X    | X    | BooleanType | StringType | X    | X    | X    | X    | X    | X    | X    | X    | X    | X
**StringType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | BinaryType | X    | StringType | DateType | TimestampType | X    | X    | X    | X    | X    | DecimalType(38, 18) | DoubleType | X
**DateType** | X    | X    | X    | X    | X    | X    | X    | X    | X    | StringType | DateType | TimestampType | X    | X    | X    | X    | X    | X    | X    | X
**TimestampType** | X    | X    | X    | X    | X    | X    | X    | X    | X    | StringType | DateType | TimestampType | X    | X    | X    | X    | X    | X    | X    | X
**ArrayType** | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | ArrayType* | X    | X    | X    | X    | X    | X    | X
**MapType** | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | MapType* | X    | X    | X    | X    | X    | X
**StructType** | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | StructType* | X    | X    | X    | X    | X
**NullType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | BinaryType | BooleanType | StringType | DateType | TimestampType | ArrayType | MapType | StructType | NullType | CalendarIntervalType | DecimalType(38, 18) | DoubleType | IntegerType
**CalendarIntervalType** | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | CalendarIntervalType | X    | X    | X
Note: ArrayType\*, MapType\*, StructType\* are castable only when the internal child types also match; otherwise, not castable
### How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15691 from gatorsmile/implicitTypeCasting.
2016-11-02 21:01:03 -07:00