Commit graph

4373 commits

Author SHA1 Message Date
Marco Gaido 88e0c7bbd5 [SPARK-24341][SQL] Support only IN subqueries with the same number of items per row
## What changes were proposed in this pull request?

Using struct types in subqueries with the `IN` clause can generate invalid plans in `RewritePredicateSubquery`. Indeed, we are not handling clearly the cases when the outer value is a struct or the output of the inner subquery is a struct.

The PR aims to make Spark's behavior the same as the one of the other RDBMS - namely Oracle and Postgres behavior were checked. So we consider valid only queries having the same number of fields in the outer value and in the subquery. This means that:

 - `(a, b) IN (select c, d from ...)` is a valid query;
 - `(a, b) IN (select (c, d) from ...)` throws an AnalysisException, as in the subquery we have only one field of type struct while in the outer value we have 2 fields;
 - `a IN (select (c, d) from ...)` - where `a` is a struct - is a valid query.

## How was this patch tested?

Added UT

Closes #21403 from mgaido91/SPARK-24313.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-07 15:43:41 +08:00
Liang-Chi Hsieh 43763629f1 [SPARK-25010][SQL] Rand/Randn should produce different values for each execution in streaming query
## What changes were proposed in this pull request?

Like Uuid in SPARK-24896, Rand and Randn expressions now produce the same results for each execution in streaming query. It doesn't make too much sense for streaming queries. We should make them produce different results as Uuid.

In this change, similar to Uuid, we assign new random seeds to Rand/Randn when returning optimized plan from `IncrementalExecution`.

Note: Different to Uuid, Rand/Randn can be created with initial seed. Because we replace this initial seed at `IncrementalExecution`, it doesn't use the initial seed anymore. For now it seems to me not a big issue for streaming query. But need to confirm with others. cc zsxwing cloud-fan

## How was this patch tested?

Added test.

Closes #21980 from viirya/SPARK-25010.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-07 14:28:14 +08:00
Kazuaki Ishizaki 4446a0b0d9 [SPARK-23914][SQL][FOLLOW-UP] refactor ArrayUnion
## What changes were proposed in this pull request?

This PR refactors `ArrayUnion` based on [this suggestion](https://github.com/apache/spark/pull/21103#discussion_r205668821).
1. Generate optimized code for all of the primitive types except `boolean`
1. Generate code using `ArrayBuilder` or `ArrayBuffer`
1. Leave only a generic path in the interpreted path

## How was this patch tested?

Existing tests

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

Closes #21937 from kiszk/SPARK-23914-follow.
2018-08-07 12:07:56 +09:00
Marco Gaido 0f3fa2f289 [SPARK-24996][SQL] Use DSL in DeclarativeAggregate
## What changes were proposed in this pull request?

The PR refactors the aggregate expressions which were not using DSL in order to simplify them.

## How was this patch tested?

NA

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21970 from mgaido91/SPARK-24996.
2018-08-06 19:46:51 -04:00
Kazuaki Ishizaki 408a3ff2c4 [SPARK-25036][SQL] Should compare ExprValue.isNull with LiteralTrue/LiteralFalse
## What changes were proposed in this pull request?

This PR fixes a comparison of `ExprValue.isNull` with `String`. `ExprValue.isNull` should be compared with `LiteralTrue` or `LiteralFalse`.

This causes the following compilation error using scala-2.12 with sbt. In addition, this code may also generate incorrect code in Spark 2.3.

```
/home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala:94: org.apache.spark.sql.catalyst.expressions.codegen.ExprValue and String are unrelated: they will most likely always compare unequal
[error] [warn]         if (eval.isNull != "true") {
[error] [warn]
[error] [warn] /home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala:126: org.apache.spark.sql.catalyst.expressions.codegen.ExprValue and String are unrelated: they will most likely never compare equal
[error] [warn]              if (eval.isNull == "true") {
[error] [warn]
[error] [warn] /home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala:133: org.apache.spark.sql.catalyst.expressions.codegen.ExprValue and String are unrelated: they will most likely never compare equal
[error] [warn]             if (eval.isNull == "true") {
[error] [warn]
[error] [warn] /home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala:90: org.apache.spark.sql.catalyst.expressions.codegen.ExprValue and String are unrelated: they will most likely never compare equal
[error] [warn]       if (inputs.map(_.isNull).forall(_ == "false")) {
[error] [warn]
```

## How was this patch tested?

Existing UTs

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

Closes #22012 from kiszk/SPARK-25036a.
2018-08-06 19:43:21 -04:00
Kazuaki Ishizaki 1a5e460762 [SPARK-23913][SQL] Add array_intersect function
## What changes were proposed in this pull request?

The PR adds the SQL function `array_intersect`. The behavior of the function is based on Presto's one.

This function returns returns an array of the elements in the intersection of array1 and array2.

Note: The order of elements in the result is not defined.

## How was this patch tested?

Added UTs

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

Closes #21102 from kiszk/SPARK-23913.
2018-08-06 23:27:57 +09:00
Dilip Biswal c1760da5dd [SPARK-25025][SQL] Remove the default value of isAll in INTERSECT/EXCEPT
## What changes were proposed in this pull request?

Having the default value of isAll in the logical plan nodes INTERSECT/EXCEPT could introduce bugs when the callers are not aware of it. This PR removes the default value and makes caller explicitly specify them.

## How was this patch tested?
This is a refactoring change. Existing tests test the functionality already.

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

Closes #22000 from dilipbiswal/SPARK-25025.
2018-08-06 06:56:36 -04:00
John Zhuge d063e3a478 [SPARK-24940][SQL] Use IntegerLiteral in ResolveCoalesceHints
## What changes were proposed in this pull request?

Follow up to fix an unmerged review comment.

## How was this patch tested?

Unit test ResolveHintsSuite.

Author: John Zhuge <jzhuge@apache.org>

Closes #21998 from jzhuge/SPARK-24940.
2018-08-06 06:41:55 -04:00
Takuya UESHIN 327bb30075 [SPARK-23911][SQL] Add aggregate function.
## What changes were proposed in this pull request?

This pr adds `aggregate` function which applies a binary operator to an initial state and all elements in the array, and reduces this to a single state. The final state is converted into the final result by applying a finish function.

```sql
> SELECT aggregate(array(1, 2, 3), (acc, x) -> acc + x);
 6
> SELECT aggregate(array(1, 2, 3), (acc, x) -> acc + x, acc -> acc * 10);
 60
```

## How was this patch tested?

Added tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #21982 from ueshin/issues/SPARK-23911/aggregate.
2018-08-05 08:58:35 +09:00
hyukjinkwon 55e3ae6930 [SPARK-25001][BUILD] Fix miscellaneous build warnings
## What changes were proposed in this pull request?

There are many warnings in the current build (for instance see https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7/4734/console).

**common**:

```
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java:237: warning: [rawtypes] found raw type: LevelDBIterator
[warn]   void closeIterator(LevelDBIterator it) throws IOException {
[warn]                      ^

[warn]   missing type arguments for generic class LevelDBIterator<T>
[warn]   where T is a type-variable:
[warn]     T extends Object declared in class LevelDBIterator
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java:151: warning: [deprecation] group() in AbstractBootstrap has been deprecated
[warn]     if (bootstrap != null && bootstrap.group() != null) {
[warn]                                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java:152: warning: [deprecation] group() in AbstractBootstrap has been deprecated
[warn]       bootstrap.group().shutdownGracefully();
[warn]                ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java:154: warning: [deprecation] childGroup() in ServerBootstrap has been deprecated
[warn]     if (bootstrap != null && bootstrap.childGroup() != null) {
[warn]                                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java:155: warning: [deprecation] childGroup() in ServerBootstrap has been deprecated
[warn]       bootstrap.childGroup().shutdownGracefully();
[warn]                ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/util/NettyUtils.java:112: warning: [deprecation] PooledByteBufAllocator(boolean,int,int,int,int,int,int,int) in PooledByteBufAllocator has been deprecated
[warn]     return new PooledByteBufAllocator(
[warn]            ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java:321: warning: [rawtypes] found raw type: Future
[warn]     public void operationComplete(Future future) throws Exception {
[warn]                                   ^

[warn]   missing type arguments for generic class Future<V>
[warn]   where V is a type-variable:
[warn]     V extends Object declared in interface Future
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java:215: warning: [rawtypes] found raw type: StreamInterceptor
[warn]           StreamInterceptor interceptor = new StreamInterceptor(this, resp.streamId, resp.byteCount,
[warn]           ^

[warn]   missing type arguments for generic class StreamInterceptor<T>
[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java:215: warning: [rawtypes] found raw type: StreamInterceptor
[warn]           StreamInterceptor interceptor = new StreamInterceptor(this, resp.streamId, resp.byteCount,
[warn]                                               ^

[warn]   missing type arguments for generic class StreamInterceptor<T>
[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java:215: warning: [unchecked] unchecked call to StreamInterceptor(MessageHandler<T>,String,long,StreamCallback) as a member of the raw type StreamInterceptor
[warn]           StreamInterceptor interceptor = new StreamInterceptor(this, resp.streamId, resp.byteCount,
[warn]                                           ^

[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java:255: warning: [rawtypes] found raw type: StreamInterceptor
[warn]         StreamInterceptor interceptor = new StreamInterceptor(this, wrappedCallback.getID(),
[warn]         ^

[warn]   missing type arguments for generic class StreamInterceptor<T>
[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java:255: warning: [rawtypes] found raw type: StreamInterceptor
[warn]         StreamInterceptor interceptor = new StreamInterceptor(this, wrappedCallback.getID(),
[warn]                                             ^

[warn]   missing type arguments for generic class StreamInterceptor<T>
[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java:255: warning: [unchecked] unchecked call to StreamInterceptor(MessageHandler<T>,String,long,StreamCallback) as a member of the raw type StreamInterceptor
[warn]         StreamInterceptor interceptor = new StreamInterceptor(this, wrappedCallback.getID(),
[warn]                                         ^

[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java:270: warning: [deprecation] transfered() in FileRegion has been deprecated
[warn]         region.transferTo(byteRawChannel, region.transfered());
[warn]                                                 ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:304: warning: [deprecation] transfered() in FileRegion has been deprecated
[warn]         region.transferTo(byteChannel, region.transfered());
[warn]                                              ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/test/java/org/apache/spark/network/ProtocolSuite.java:119: warning: [deprecation] transfered() in FileRegion has been deprecated
[warn]       while (in.transfered() < in.count()) {
[warn]                ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/test/java/org/apache/spark/network/ProtocolSuite.java:120: warning: [deprecation] transfered() in FileRegion has been deprecated
[warn]         in.transferTo(channel, in.transfered());
[warn]                                  ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java:80: warning: [static] static method should be qualified by type name, Murmur3_x86_32, instead of by an expression
[warn]     Assert.assertEquals(-300363099, hasher.hashUnsafeWords(bytes, offset, 16, 42));
[warn]                                           ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java:84: warning: [static] static method should be qualified by type name, Murmur3_x86_32, instead of by an expression
[warn]     Assert.assertEquals(-1210324667, hasher.hashUnsafeWords(bytes, offset, 16, 42));
[warn]                                            ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java:88: warning: [static] static method should be qualified by type name, Murmur3_x86_32, instead of by an expression
[warn]     Assert.assertEquals(-634919701, hasher.hashUnsafeWords(bytes, offset, 16, 42));
[warn]                                           ^
```

**launcher**:

```
[warn] Pruning sources from previous analysis, due to incompatible CompileSetup.
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java:31: warning: [rawtypes] found raw type: AbstractLauncher
[warn] public abstract class AbstractLauncher<T extends AbstractLauncher> {
[warn]                                                  ^
[warn]   missing type arguments for generic class AbstractLauncher<T>
[warn]   where T is a type-variable:
[warn]     T extends AbstractLauncher declared in class AbstractLauncher
```

**core**:

```
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/main/scala/org/apache/spark/api/r/RBackend.scala:99: method group in class AbstractBootstrap is deprecated: see corresponding Javadoc for more information.
[warn]     if (bootstrap != null && bootstrap.group() != null) {
[warn]                                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/main/scala/org/apache/spark/api/r/RBackend.scala💯 method group in class AbstractBootstrap is deprecated: see corresponding Javadoc for more information.
[warn]       bootstrap.group().shutdownGracefully()
[warn]                 ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/main/scala/org/apache/spark/api/r/RBackend.scala:102: method childGroup in class ServerBootstrap is deprecated: see corresponding Javadoc for more information.
[warn]     if (bootstrap != null && bootstrap.childGroup() != null) {
[warn]                                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/main/scala/org/apache/spark/api/r/RBackend.scala:103: method childGroup in class ServerBootstrap is deprecated: see corresponding Javadoc for more information.
[warn]       bootstrap.childGroup().shutdownGracefully()
[warn]                 ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala:151: reflective access of structural type member method getData should be enabled
[warn] by making the implicit value scala.language.reflectiveCalls visible.
[warn] This can be achieved by adding the import clause 'import scala.language.reflectiveCalls'
[warn] or by setting the compiler option -language:reflectiveCalls.
[warn] See the Scaladoc for value scala.language.reflectiveCalls for a discussion
[warn] why the feature should be explicitly enabled.
[warn]       val rdd = sc.parallelize(1 to 1).map(concreteObject.getData)
[warn]                                                           ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala:175: reflective access of structural type member value innerObject2 should be enabled
[warn] by making the implicit value scala.language.reflectiveCalls visible.
[warn]       val rdd = sc.parallelize(1 to 1).map(concreteObject.innerObject2.getData)
[warn]                                                           ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala:175: reflective access of structural type member method getData should be enabled
[warn] by making the implicit value scala.language.reflectiveCalls visible.
[warn]       val rdd = sc.parallelize(1 to 1).map(concreteObject.innerObject2.getData)
[warn]                                                                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/LocalSparkContext.scala:32: constructor Slf4JLoggerFactory in class Slf4JLoggerFactory is deprecated: see corresponding Javadoc for more information.
[warn]     InternalLoggerFactory.setDefaultFactory(new Slf4JLoggerFactory())
[warn]                                             ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:218: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]         assert(wrapper.stageAttemptId === stages.head.attemptId)
[warn]                                                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:261: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       stageAttemptId = stages.head.attemptId))
[warn]                                    ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:287: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       stageAttemptId = stages.head.attemptId))
[warn]                                    ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:471: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       stageAttemptId = stages.last.attemptId))
[warn]                                    ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:966: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]     listener.onTaskStart(SparkListenerTaskStart(dropped.stageId, dropped.attemptId, task))
[warn]                                                                          ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:972: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]     listener.onTaskEnd(SparkListenerTaskEnd(dropped.stageId, dropped.attemptId,
[warn]                                                                      ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:976: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       .taskSummary(dropped.stageId, dropped.attemptId, Array(0.25d, 0.50d, 0.75d))
[warn]                                             ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:1146: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       SparkListenerTaskEnd(stage1.stageId, stage1.attemptId, "taskType", Success, tasks(1), null))
[warn]                                                   ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:1150: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       SparkListenerTaskEnd(stage1.stageId, stage1.attemptId, "taskType", Success, tasks(0), null))
[warn]                                                   ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala:197: method transfered in trait FileRegion is deprecated: see corresponding Javadoc for more information.
[warn]     while (region.transfered() < region.count()) {
[warn]                   ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala:198: method transfered in trait FileRegion is deprecated: see corresponding Javadoc for more information.
[warn]       region.transferTo(byteChannel, region.transfered())
[warn]                                             ^
```

**sql**:

```
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala:534: abstract type T is unchecked since it is eliminated by erasure
[warn]       assert(partitioning.isInstanceOf[T])
[warn]                                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala:534: abstract type T is unchecked since it is eliminated by erasure
[warn]       assert(partitioning.isInstanceOf[T])
[warn]             ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala:323: inferred existential type Option[Class[_$1]]( forSome { type _$1 }), which cannot be expressed by wildcards,  should be enabled
[warn] by making the implicit value scala.language.existentials visible.
[warn] This can be achieved by adding the import clause 'import scala.language.existentials'
[warn] or by setting the compiler option -language:existentials.
[warn] See the Scaladoc for value scala.language.existentials for a discussion
[warn] why the feature should be explicitly enabled.
[warn]       val optClass = Option(collectionCls)
[warn]                            ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:226: warning: [deprecation] ParquetFileReader(Configuration,FileMetaData,Path,List<BlockMetaData>,List<ColumnDescriptor>) in ParquetFileReader has been deprecated
[warn]     this.reader = new ParquetFileReader(
[warn]                   ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:178: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             (descriptor.getType() == PrimitiveType.PrimitiveTypeName.INT32 ||
[warn]                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:179: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             (descriptor.getType() == PrimitiveType.PrimitiveTypeName.INT64  &&
[warn]                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:181: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             descriptor.getType() == PrimitiveType.PrimitiveTypeName.FLOAT ||
[warn]                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:182: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             descriptor.getType() == PrimitiveType.PrimitiveTypeName.DOUBLE ||
[warn]                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:183: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             descriptor.getType() == PrimitiveType.PrimitiveTypeName.BINARY))) {
[warn]                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:198: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]         switch (descriptor.getType()) {
[warn]                           ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:221: warning: [deprecation] getTypeLength() in ColumnDescriptor has been deprecated
[warn]             readFixedLenByteArrayBatch(rowId, num, column, descriptor.getTypeLength());
[warn]                                                                      ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:224: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             throw new IOException("Unsupported type: " + descriptor.getType());
[warn]                                                                    ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:246: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]       descriptor.getType().toString(),
[warn]                 ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:258: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]     switch (descriptor.getType()) {
[warn]                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:384: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]         throw new UnsupportedOperationException("Unsupported type: " + descriptor.getType());
[warn]                                                                                  ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java:458: warning: [static] static variable should be qualified by type name, BaseRepeatedValueVector, instead of by an expression
[warn]       int index = rowId * accessor.OFFSET_WIDTH;
[warn]                                   ^
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java:460: warning: [static] static variable should be qualified by type name, BaseRepeatedValueVector, instead of by an expression
[warn]       int end = offsets.getInt(index + accessor.OFFSET_WIDTH);
[warn]                                                ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/BenchmarkQueryTest.scala:57: a pure expression does nothing in statement position; you may be omitting necessary parentheses
[warn]       case s => s
[warn]                 ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala:182: inferred existential type org.apache.parquet.column.statistics.Statistics[?0]( forSome { type ?0 <: Comparable[?0] }), which cannot be expressed by wildcards,  should be enabled
[warn] by making the implicit value scala.language.existentials visible.
[warn] This can be achieved by adding the import clause 'import scala.language.existentials'
[warn] or by setting the compiler option -language:existentials.
[warn] See the Scaladoc for value scala.language.existentials for a discussion
[warn] why the feature should be explicitly enabled.
[warn]                 val columnStats = oneBlockColumnMeta.getStatistics
[warn]                                                      ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala:146: implicit conversion method conv should be enabled
[warn] by making the implicit value scala.language.implicitConversions visible.
[warn] This can be achieved by adding the import clause 'import scala.language.implicitConversions'
[warn] or by setting the compiler option -language:implicitConversions.
[warn] See the Scaladoc for value scala.language.implicitConversions for a discussion
[warn] why the feature should be explicitly enabled.
[warn]     implicit def conv(x: (Int, Long)): KV = KV(x._1, x._2)
[warn]                  ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/shuffle/ContinuousShuffleSuite.scala:48: implicit conversion method unsafeRow should be enabled
[warn] by making the implicit value scala.language.implicitConversions visible.
[warn]   private implicit def unsafeRow(value: Int) = {
[warn]                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala:178: method getType in class ColumnDescriptor is deprecated: see corresponding Javadoc for more information.
[warn]                 assert(oneFooter.getFileMetaData.getSchema.getColumns.get(0).getType() ===
[warn]                                                                              ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala:154: method readAllFootersInParallel in object ParquetFileReader is deprecated: see corresponding Javadoc for more information.
[warn]     ParquetFileReader.readAllFootersInParallel(configuration, fs.getFileStatus(path)).asScala.toSeq
[warn]                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/hive/src/test/java/org/apache/spark/sql/hive/test/Complex.java:679: warning: [cast] redundant cast to Complex
[warn]     Complex typedOther = (Complex)other;
[warn]                          ^
```

**mllib**:

```
[warn] Pruning sources from previous analysis, due to incompatible CompileSetup.
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala:597: match may not be exhaustive.
[warn] It would fail on the following inputs: None, Some((x: Tuple2[?, ?] forSome x not in (?, ?)))
[warn]     val df = dfs.find {
[warn]                       ^
```

This PR does not target fix all of them since some look pretty tricky to fix and there look too many warnings including false positive (like deprecated API but it's used in its test, etc.)

## How was this patch tested?

Existing tests should cover this.

Author: hyukjinkwon <gurwls223@apache.org>

Closes #21975 from HyukjinKwon/remove-build-warnings.
2018-08-04 11:52:49 -05:00
Wenchen Fan 684c719cc0 [SPARK-23915][SQL][FOLLOWUP] Add array_except function
## What changes were proposed in this pull request?

simplify the codegen:
1. only do real codegen if the type can be specialized by the hash set
2. change the null handling. Before: track the nullElementIndex, and create a new ArrayData to insert the null in the middle. After: track the nullElementIndex, put a null placeholder in the ArrayBuilder, at the end create ArrayData from ArrayBuilder directly.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21966 from cloud-fan/minor2.
2018-08-04 16:35:14 +09:00
Takuya UESHIN 0ecc132d6b [SPARK-23909][SQL] Add filter function.
## What changes were proposed in this pull request?

This pr adds `filter` function which filters the input array using the given predicate.

```sql
> SELECT filter(array(1, 2, 3), x -> x % 2 == 1);
 array(1, 3)
```

## How was this patch tested?

Added tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #21965 from ueshin/issues/SPARK-23909/filter.
2018-08-04 16:08:53 +09:00
John Zhuge 36ea55e97e [SPARK-24940][SQL] Coalesce and Repartition Hint for SQL Queries
## What changes were proposed in this pull request?

Many Spark SQL users in my company have asked for a way to control the number of output files in Spark SQL. The users prefer not to use function repartition(n) or coalesce(n, shuffle) that require them to write and deploy Scala/Java/Python code. We propose adding the following Hive-style Coalesce and Repartition Hint to Spark SQL:
```
... SELECT /*+ COALESCE(numPartitions) */ ...
... SELECT /*+ REPARTITION(numPartitions) */ ...
```
Multiple such hints are allowed. Multiple nodes are inserted into the logical plan, and the optimizer will pick the leftmost hint.
```
INSERT INTO s SELECT /*+ REPARTITION(100), COALESCE(500), COALESCE(10) */ * FROM t

== Logical Plan ==
'InsertIntoTable 'UnresolvedRelation `s`, false, false
+- 'UnresolvedHint REPARTITION, [100]
   +- 'UnresolvedHint COALESCE, [500]
      +- 'UnresolvedHint COALESCE, [10]
         +- 'Project [*]
            +- 'UnresolvedRelation `t`

== Optimized Logical Plan ==
InsertIntoHadoopFsRelationCommand ...
+- Repartition 100, true
   +- HiveTableRelation ...
```

## How was this patch tested?

All unit tests. Manual tests using explain.

Author: John Zhuge <jzhuge@apache.org>

Closes #21911 from jzhuge/SPARK-24940.
2018-08-04 02:27:15 -04:00
Dilip Biswal 19a4531913 [SPARK-24997][SQL] Enable support of MINUS ALL
## What changes were proposed in this pull request?
Enable support for MINUS ALL which was gated at AstBuilder.

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

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

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

Closes #21963 from dilipbiswal/minus-all.
2018-08-02 22:45:10 -07:00
Dilip Biswal 73dd6cf9b5 [SPARK-24966][SQL] Implement precedence rules for set operations.
## What changes were proposed in this pull request?

Currently the set operations INTERSECT, UNION and EXCEPT are assigned the same precedence. This PR fixes the problem by giving INTERSECT  higher precedence than UNION and EXCEPT. UNION and EXCEPT operators are evaluated in the order in which they appear in the query from left to right.

This results in change in behavior because of the change in order of evaluations of set operators in a query. The old behavior is still preserved under a newly added config parameter.

Query `:`
```
SELECT * FROM t1
UNION
SELECT * FROM t2
EXCEPT
SELECT * FROM t3
INTERSECT
SELECT * FROM t4
```
Parsed plan before the change `:`
```
== Parsed Logical Plan ==
'Intersect false
:- 'Except false
:  :- 'Distinct
:  :  +- 'Union
:  :     :- 'Project [*]
:  :     :  +- 'UnresolvedRelation `t1`
:  :     +- 'Project [*]
:  :        +- 'UnresolvedRelation `t2`
:  +- 'Project [*]
:     +- 'UnresolvedRelation `t3`
+- 'Project [*]
   +- 'UnresolvedRelation `t4`
```
Parsed plan after the change `:`
```
== Parsed Logical Plan ==
'Except false
:- 'Distinct
:  +- 'Union
:     :- 'Project [*]
:     :  +- 'UnresolvedRelation `t1`
:     +- 'Project [*]
:        +- 'UnresolvedRelation `t2`
+- 'Intersect false
   :- 'Project [*]
   :  +- 'UnresolvedRelation `t3`
   +- 'Project [*]
      +- 'UnresolvedRelation `t4`
```
## How was this patch tested?
Added tests in PlanParserSuite, SQLQueryTestSuite.

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

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

Closes #21941 from dilipbiswal/SPARK-24966.
2018-08-02 22:04:17 -07:00
Gengliang Wang 7cf16a7fa4 [SPARK-24773] Avro: support logical timestamp type with different precisions
## What changes were proposed in this pull request?

Support reading/writing Avro logical timestamp type with different precisions
https://avro.apache.org/docs/1.8.2/spec.html#Timestamp+%28millisecond+precision%29

To specify the output timestamp type, use Dataframe option `outputTimestampType`  or SQL config `spark.sql.avro.outputTimestampType`.  The supported values are
* `TIMESTAMP_MICROS`
* `TIMESTAMP_MILLIS`

The default output type is `TIMESTAMP_MICROS`
## How was this patch tested?

Unit test

Author: Gengliang Wang <gengliang.wang@databricks.com>

Closes #21935 from gengliangwang/avro_timestamp.
2018-08-03 08:32:08 +08:00
Kazuaki Ishizaki bbdcc3bf61 [SPARK-22219][SQL] Refactor code to get a value for "spark.sql.codegen.comments"
## What changes were proposed in this pull request?

This PR refactors code to get a value for "spark.sql.codegen.comments" by avoiding `SparkEnv.get.conf`. This PR uses `SQLConf.get.codegenComments` since `SQLConf.get` always returns an instance of `SQLConf`.

## How was this patch tested?

Added test case to `DebuggingSuite`

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

Closes #19449 from kiszk/SPARK-22219.
2018-08-02 18:19:04 -05:00
Takuya UESHIN 02f967795b [SPARK-23908][SQL] Add transform function.
## What changes were proposed in this pull request?

This pr adds `transform` function which transforms elements in an array using the function.
Optionally we can take the index of each element as the second argument.

```sql
> SELECT transform(array(1, 2, 3), x -> x + 1);
 array(2, 3, 4)
> SELECT transform(array(1, 2, 3), (x, i) -> x + i);
 array(1, 3, 5)
```

## How was this patch tested?

Added tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #21954 from ueshin/issues/SPARK-23908/transform.
2018-08-02 13:00:33 -07:00
Wenchen Fan f04cd67094 [MINOR] remove dead code in ExpressionEvalHelper
## What changes were proposed in this pull request?

This addresses https://github.com/apache/spark/pull/21236/files#r207078480

both https://github.com/apache/spark/pull/21236 and https://github.com/apache/spark/pull/21838 add a InternalRow result check to ExpressionEvalHelper and becomes duplicated.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21958 from cloud-fan/minor.
2018-08-02 09:26:27 -05:00
Kaya Kupferschmidt 7be6fc3c77 [SPARK-24742] Fix NullPointerexception in Field Metadata
## What changes were proposed in this pull request?

This pull request provides a fix for SPARK-24742: SQL Field MetaData was throwing an Exception in the hashCode method when a "null" Metadata was added via "putNull"

## How was this patch tested?

A new unittest is provided in org/apache/spark/sql/types/MetadataSuite.scala

Author: Kaya Kupferschmidt <k.kupferschmidt@dimajix.de>

Closes #21722 from kupferk/SPARK-24742.
2018-08-02 09:22:21 -05:00
Xiao Li 46110a589f [SPARK-24865][FOLLOW-UP] Remove AnalysisBarrier LogicalPlan Node
## What changes were proposed in this pull request?
Remove the AnalysisBarrier LogicalPlan node, which is useless now.

## How was this patch tested?
N/A

Author: Xiao Li <gatorsmile@gmail.com>

Closes #21962 from gatorsmile/refactor2.
2018-08-02 22:20:41 +08:00
Xiao Li 166f346185 [SPARK-24957][SQL][FOLLOW-UP] Clean the code for AVERAGE
## What changes were proposed in this pull request?
This PR is to refactor the code in AVERAGE by dsl.

## How was this patch tested?
N/A

Author: Xiao Li <gatorsmile@gmail.com>

Closes #21951 from gatorsmile/refactor1.
2018-08-01 23:00:17 -07:00
Kazuaki Ishizaki 95a9d5e3a5 [SPARK-23915][SQL] Add array_except function
## What changes were proposed in this pull request?

The PR adds the SQL function `array_except`. The behavior of the function is based on Presto's one.

This function returns returns an array of the elements in array1 but not in array2.

Note: The order of elements in the result is not defined.

## How was this patch tested?

Added UTs.

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

Closes #21103 from kiszk/SPARK-23915.
2018-08-02 02:52:30 +08:00
Reynold Xin 1efffb7993 [SPARK-24982][SQL] UDAF resolution should not throw AssertionError
## What changes were proposed in this pull request?
When user calls anUDAF with the wrong number of arguments, Spark previously throws an AssertionError, which is not supposed to be a user-facing exception.  This patch updates it to throw AnalysisException instead, so it is consistent with a regular UDF.

## How was this patch tested?
Updated test case udaf.sql.

Author: Reynold Xin <rxin@databricks.com>

Closes #21938 from rxin/SPARK-24982.
2018-08-01 00:15:31 -07:00
Reynold Xin 1f7e22c72c [SPARK-24951][SQL] Table valued functions should throw AnalysisException
## What changes were proposed in this pull request?
Previously TVF resolution could throw IllegalArgumentException if the data type is null type. This patch replaces that exception with AnalysisException, enriched with positional information, to improve error message reporting and to be more consistent with rest of Spark SQL.

## How was this patch tested?
Updated the test case in table-valued-functions.sql.out, which is how I identified this problem in the first place.

Author: Reynold Xin <rxin@databricks.com>

Closes #21934 from rxin/SPARK-24951.
2018-07-31 22:25:40 -07:00
DB Tsai 5f3441e542 [SPARK-24893][SQL] Remove the entire CaseWhen if all the outputs are semantic equivalence
## What changes were proposed in this pull request?

Similar to SPARK-24890, if all the outputs of `CaseWhen` are semantic equivalence, `CaseWhen` can be removed.

## How was this patch tested?

Tests added.

Author: DB Tsai <d_tsai@apple.com>

Closes #21852 from dbtsai/short-circuit-when.
2018-08-01 10:31:02 +08:00
Mauro Palsgraaf 4ac2126bc6 [SPARK-24536] Validate that an evaluated limit clause cannot be null
## What changes were proposed in this pull request?

It proposes a version in which nullable expressions are not valid in the limit clause

## How was this patch tested?

It was tested with unit and e2e tests.

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

Author: Mauro Palsgraaf <mauropalsgraaf@hotmail.com>

Closes #21807 from mauropalsgraaf/SPARK-24536.
2018-07-31 08:18:08 -07:00
maryannxue b4fd75fb9b [SPARK-24972][SQL] PivotFirst could not handle pivot columns of complex types
## What changes were proposed in this pull request?

When the pivot column is of a complex type, the eval() result will be an UnsafeRow, while the keys of the HashMap for column value matching is a GenericInternalRow. As a result, there will be no match and the result will always be empty.
So for a pivot column of complex-types, we should:
1) If the complex-type is not comparable (orderable), throw an Exception. It cannot be a pivot column.
2) Otherwise, if it goes through the `PivotFirst` code path, `PivotFirst` should use a TreeMap instead of HashMap for such columns.

This PR has also reverted the walk-around in Analyzer that had been introduced to avoid this `PivotFirst` issue.

## How was this patch tested?

Added UT.

Author: maryannxue <maryannxue@apache.org>

Closes #21926 from maryannxue/pivot_followup.
2018-07-30 23:43:53 -07:00
Maxim Gekk d20c10fdf3 [SPARK-24952][SQL] Support LZMA2 compression by Avro datasource
## What changes were proposed in this pull request?

In the PR, I propose to support `LZMA2` (`XZ`) and `BZIP2` compressions by `AVRO` datasource  in write since the codecs may have better characteristics like compression ratio and speed comparing to already supported `snappy` and `deflate` codecs.

## How was this patch tested?

It was tested manually and by an existing test which was extended to check the `xz` and `bzip2` compressions.

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21902 from MaxGekk/avro-xz-bzip2.
2018-07-31 09:12:57 +08:00
Reynold Xin abbb4ab4d8 [SPARK-24865][SQL] Remove AnalysisBarrier addendum
## What changes were proposed in this pull request?
I didn't want to pollute the diff in the previous PR and left some TODOs. This is a follow-up to address those TODOs.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #21896 from rxin/SPARK-24865-addendum.
2018-07-30 14:05:45 -07:00
Takeshi Yamamuro 47d84e4d0e [SPARK-22814][SQL] Support Date/Timestamp in a JDBC partition column
## What changes were proposed in this pull request?
This pr supported Date/Timestamp in a JDBC partition column (a numeric column is only supported in the master). This pr also modified code to verify a partition column type;
```
val jdbcTable = spark.read
 .option("partitionColumn", "text")
 .option("lowerBound", "aaa")
 .option("upperBound", "zzz")
 .option("numPartitions", 2)
 .jdbc("jdbc:postgresql:postgres", "t", options)

// with this pr
org.apache.spark.sql.AnalysisException: Partition column type should be numeric, date, or timestamp, but string found.;
  at org.apache.spark.sql.execution.datasources.jdbc.JDBCRelation$.verifyAndGetNormalizedPartitionColumn(JDBCRelation.scala:165)
  at org.apache.spark.sql.execution.datasources.jdbc.JDBCRelation$.columnPartition(JDBCRelation.scala:85)
  at org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider.createRelation(JdbcRelationProvider.scala:36)
  at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:317)

// without this pr
java.lang.NumberFormatException: For input string: "aaa"
  at java.lang.NumberFormatException.forInputString(NumberFormatException.java:65)
  at java.lang.Long.parseLong(Long.java:589)
  at java.lang.Long.parseLong(Long.java:631)
  at scala.collection.immutable.StringLike$class.toLong(StringLike.scala:277)
```

Closes #19999

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21834 from maropu/SPARK-22814.
2018-07-30 07:42:00 -07:00
Marco Gaido 85505fc8a5 [SPARK-24957][SQL] Average with decimal followed by aggregation returns wrong result
## What changes were proposed in this pull request?

When we do an average, the result is computed dividing the sum of the values by their count. In the case the result is a DecimalType, the way we are casting/managing the precision and scale is not really optimized and it is not coherent with what we do normally.

In particular, a problem can happen when the `Divide` operand returns a result which contains a precision and scale different by the ones which are expected as output of the `Divide` operand. In the case reported in the JIRA, for instance, the result of the `Divide` operand is a `Decimal(38, 36)`, while the output data type for `Divide` is 38, 22. This is not an issue when the `Divide` is followed by a `CheckOverflow` or a `Cast` to the right data type, as these operations return a decimal with the defined precision and scale. Despite in the `Average` operator we do have a `Cast`, this may be bypassed if the result of `Divide` is the same type which it is casted to, hence the issue reported in the JIRA may arise.

The PR proposes to use the normal rules/handling of the arithmetic operators with Decimal data type, so we both reuse the existing code (having a single logic for operations between decimals) and we fix this problem as the result is always guarded by `CheckOverflow`.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21910 from mgaido91/SPARK-24957.
2018-07-30 20:53:45 +08:00
Dilip Biswal 65a4bc143a [SPARK-21274][SQL] Implement INTERSECT ALL clause
## What changes were proposed in this pull request?
Implements INTERSECT ALL clause through query rewrites using existing operators in Spark.  Please refer to [Link](https://drive.google.com/open?id=1nyW0T0b_ajUduQoPgZLAsyHK8s3_dko3ulQuxaLpUXE) for the design.

Input Query
``` SQL
SELECT c1 FROM ut1 INTERSECT ALL SELECT c1 FROM ut2
```
Rewritten Query
```SQL
   SELECT c1
    FROM (
         SELECT replicate_row(min_count, c1)
         FROM (
              SELECT c1,
                     IF (vcol1_cnt > vcol2_cnt, vcol2_cnt, vcol1_cnt) AS min_count
              FROM (
                   SELECT   c1, count(vcol1) as vcol1_cnt, count(vcol2) as vcol2_cnt
                   FROM (
                        SELECT c1, true as vcol1, null as vcol2 FROM ut1
                        UNION ALL
                        SELECT c1, null as vcol1, true as vcol2 FROM ut2
                        ) AS union_all
                   GROUP BY c1
                   HAVING vcol1_cnt >= 1 AND vcol2_cnt >= 1
                  )
              )
          )
```

## How was this patch tested?
Added test cases in SQLQueryTestSuite, DataFrameSuite, SetOperationSuite

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

Closes #21886 from dilipbiswal/dkb_intersect_all_final.
2018-07-29 22:11:01 -07:00
Chris Martin c5b8d54c61 [SPARK-24950][SQL] DateTimeUtilsSuite daysToMillis and millisToDays fails w/java 8 181-b13
## What changes were proposed in this pull request?

- Update DateTimeUtilsSuite so that when testing roundtripping in daysToMillis and millisToDays multiple skipdates can be specified.
- Updated test so that both new years eve 2014 and new years day 2015 are skipped for kiribati time zones.  This is necessary as java versions pre 181-b13 considered new years day 2015 to be skipped while susequent versions corrected this to new years eve.

## How was this patch tested?
Unit tests

Author: Chris Martin <chris@cmartinit.co.uk>

Closes #21901 from d80tb7/SPARK-24950_datetimeUtilsSuite_failures.
2018-07-28 10:40:10 -05:00
Reynold Xin 34ebcc6b52 [MINOR] Improve documentation for HiveStringType's
The diff should be self-explanatory.

Author: Reynold Xin <rxin@databricks.com>

Closes #21897 from rxin/hivestringtypedoc.
2018-07-27 15:34:06 -07:00
Dilip Biswal 10f1f19659 [SPARK-21274][SQL] Implement EXCEPT ALL clause.
## What changes were proposed in this pull request?
Implements EXCEPT ALL clause through query rewrites using existing operators in Spark. In this PR, an internal UDTF (replicate_rows) is added to aid in preserving duplicate rows. Please refer to [Link](https://drive.google.com/open?id=1nyW0T0b_ajUduQoPgZLAsyHK8s3_dko3ulQuxaLpUXE) for the design.

**Note** This proposed UDTF is kept as a internal function that is purely used to aid with this particular rewrite to give us flexibility to change to a more generalized UDTF in future.

Input Query
``` SQL
SELECT c1 FROM ut1 EXCEPT ALL SELECT c1 FROM ut2
```
Rewritten Query
```SQL
SELECT c1
    FROM (
     SELECT replicate_rows(sum_val, c1)
       FROM (
         SELECT c1, sum_val
           FROM (
             SELECT c1, sum(vcol) AS sum_val
               FROM (
                 SELECT 1L as vcol, c1 FROM ut1
                 UNION ALL
                 SELECT -1L as vcol, c1 FROM ut2
              ) AS union_all
            GROUP BY union_all.c1
          )
        WHERE sum_val > 0
       )
   )
```

## How was this patch tested?
Added test cases in SQLQueryTestSuite, DataFrameSuite and SetOperationSuite

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

Closes #21857 from dilipbiswal/dkb_except_all_final.
2018-07-27 13:47:33 -07:00
Maxim Gekk 0a0f68bae6 [SPARK-24881][SQL] New Avro option - compression
## What changes were proposed in this pull request?

In the PR, I added new option for Avro datasource - `compression`. The option allows to specify compression codec for saved Avro files. This option is similar to `compression` option in another datasources like `JSON` and `CSV`.

Also I added the SQL configs `spark.sql.avro.compression.codec` and `spark.sql.avro.deflate.level`. I put the configs into `SQLConf`. If the `compression` option is not specified by an user, the first SQL config is taken into account.

## How was this patch tested?

I added new test which read meta info from written avro files and checks `avro.codec` property.

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21837 from MaxGekk/avro-compression.
2018-07-28 00:11:32 +08:00
pkuwm ef6c8395c4 [SPARK-23928][SQL] Add shuffle collection function.
## What changes were proposed in this pull request?

This PR adds a new collection function: shuffle. It generates a random permutation of the given array. This implementation uses the "inside-out" version of Fisher-Yates algorithm.

## How was this patch tested?

New tests are added to CollectionExpressionsSuite.scala and DataFrameFunctionsSuite.scala.

Author: Takuya UESHIN <ueshin@databricks.com>
Author: pkuwm <ihuizhi.lu@gmail.com>

Closes #21802 from ueshin/issues/SPARK-23928/shuffle.
2018-07-27 23:02:48 +09:00
Reynold Xin e6e9031d7b [SPARK-24865] Remove AnalysisBarrier
## What changes were proposed in this pull request?
AnalysisBarrier was introduced in SPARK-20392 to improve analysis speed (don't re-analyze nodes that have already been analyzed).

Before AnalysisBarrier, we already had some infrastructure in place, with analysis specific functions (resolveOperators and resolveExpressions). These functions do not recursively traverse down subplans that are already analyzed (with a mutable boolean flag _analyzed). The issue with the old system was that developers started using transformDown, which does a top-down traversal of the plan tree, because there was not top-down resolution function, and as a result analyzer performance became pretty bad.

In order to fix the issue in SPARK-20392, AnalysisBarrier was introduced as a special node and for this special node, transform/transformUp/transformDown don't traverse down. However, the introduction of this special node caused a lot more troubles than it solves. This implicit node breaks assumptions and code in a few places, and it's hard to know when analysis barrier would exist, and when it wouldn't. Just a simple search of AnalysisBarrier in PR discussions demonstrates it is a source of bugs and additional complexity.

Instead, this pull request removes AnalysisBarrier and reverts back to the old approach. We added infrastructure in tests that fail explicitly if transform methods are used in the analyzer.

## How was this patch tested?
Added a test suite AnalysisHelperSuite for testing the resolve* methods and transform* methods.

Author: Reynold Xin <rxin@databricks.com>
Author: Xiao Li <gatorsmile@gmail.com>

Closes #21822 from rxin/SPARK-24865.
2018-07-27 14:29:05 +08:00
maryannxue 5ed7660d14 [SPARK-24802][SQL][FOLLOW-UP] Add a new config for Optimization Rule Exclusion
## What changes were proposed in this pull request?

This is an extension to the original PR, in which rule exclusion did not work for classes derived from Optimizer, e.g., SparkOptimizer.
To solve this issue, Optimizer and its derived classes will define/override `defaultBatches` and `nonExcludableRules` in order to define its default rule set as well as rules that cannot be excluded by the SQL config. In the meantime, Optimizer's `batches` method is dedicated to the rule exclusion logic and is defined "final".

## How was this patch tested?

Added UT.

Author: maryannxue <maryannxue@apache.org>

Closes #21876 from maryannxue/rule-exclusion.
2018-07-26 11:06:23 -07:00
Takuya UESHIN c9b233d414 [SPARK-24878][SQL] Fix reverse function for array type of primitive type containing null.
## What changes were proposed in this pull request?

If we use `reverse` function for array type of primitive type containing `null` and the child array is `UnsafeArrayData`, the function returns a wrong result because `UnsafeArrayData` doesn't define the behavior of re-assignment, especially we can't set a valid value after we set `null`.

## How was this patch tested?

Added some tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #21830 from ueshin/issues/SPARK-24878/fix_reverse.
2018-07-26 15:06:13 +08:00
Koert Kuipers 17f469bc80 [SPARK-24860][SQL] Support setting of partitionOverWriteMode in output options for writing DataFrame
## What changes were proposed in this pull request?

Besides spark setting spark.sql.sources.partitionOverwriteMode also allow setting partitionOverWriteMode per write

## How was this patch tested?

Added unit test in InsertSuite

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

Author: Koert Kuipers <koert@tresata.com>

Closes #21818 from koertkuipers/feat-partition-overwrite-mode-per-write.
2018-07-25 13:06:03 -07:00
Maxim Gekk 2f77616e1d [SPARK-24849][SPARK-24911][SQL] Converting a value of StructType to a DDL string
## What changes were proposed in this pull request?

In the PR, I propose to extend the `StructType`/`StructField` classes by new method `toDDL` which converts a value of the `StructType`/`StructField` type to a string formatted in DDL style. The resulted string can be used in a table creation.

The `toDDL` method of `StructField` is reused in `SHOW CREATE TABLE`. In this way the PR fixes the bug of unquoted names of nested fields.

## How was this patch tested?

I add a test for checking the new method and 2 round trip tests: `fromDDL` -> `toDDL` and `toDDL` -> `fromDDL`

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21803 from MaxGekk/to-ddl.
2018-07-25 11:09:12 -07:00
Yuming Wang 7a5fd4a91e [SPARK-18874][SQL][FOLLOW-UP] Improvement type mismatched message
## What changes were proposed in this pull request?
Improvement `IN` predicate type mismatched message:
```sql
Mismatched columns:
[(, t, 4, ., `, t, 4, a, `, :, d, o, u, b, l, e, ,,  , t, 5, ., `, t, 5, a, `, :, d, e, c, i, m, a, l, (, 1, 8, ,, 0, ), ), (, t, 4, ., `, t, 4, c, `, :, s, t, r, i, n, g, ,,  , t, 5, ., `, t, 5, c, `, :, b, i, g, i, n, t, )]
```
After this patch:
```sql
Mismatched columns:
[(t4.`t4a`:double, t5.`t5a`:decimal(18,0)), (t4.`t4c`:string, t5.`t5c`:bigint)]
```

## How was this patch tested?

unit tests

Author: Yuming Wang <yumwang@ebay.com>

Closes #21863 from wangyum/SPARK-18874.
2018-07-24 23:59:13 -07:00
Dilip Biswal afb0627536 [SPARK-23957][SQL] Sorts in subqueries are redundant and can be removed
## What changes were proposed in this pull request?
Thanks to henryr for the original idea at https://github.com/apache/spark/pull/21049

Description from the original PR :
Subqueries (at least in SQL) have 'bag of tuples' semantics. Ordering
them is therefore redundant (unless combined with a limit).

This patch removes the top sort operators from the subquery plans.

This closes https://github.com/apache/spark/pull/21049.

## How was this patch tested?
Added test cases in SubquerySuite to cover in, exists and scalar subqueries.

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

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

Closes #21853 from dilipbiswal/SPARK-23957.
2018-07-24 20:46:27 -07:00
DB Tsai d4c3415894 [SPARK-24890][SQL] Short circuiting the if condition when trueValue and falseValue are the same
## What changes were proposed in this pull request?

When `trueValue` and `falseValue` are semantic equivalence, the condition expression in `if` can be removed to avoid extra computation in runtime.

## How was this patch tested?

Test added.

Author: DB Tsai <d_tsai@apple.com>

Closes #21848 from dbtsai/short-circuit-if.
2018-07-24 20:21:11 -07:00
maryannxue c26b092169 [SPARK-24891][SQL] Fix HandleNullInputsForUDF rule
## What changes were proposed in this pull request?

The HandleNullInputsForUDF would always add a new `If` node every time it is applied. That would cause a difference between the same plan being analyzed once and being analyzed twice (or more), thus raising issues like plan not matched in the cache manager. The solution is to mark the arguments as null-checked, which is to add a "KnownNotNull" node above those arguments, when adding the UDF under an `If` node, because clearly the UDF will not be called when any of those arguments is null.

## How was this patch tested?

Add new tests under sql/UDFSuite and AnalysisSuite.

Author: maryannxue <maryannxue@apache.org>

Closes #21851 from maryannxue/spark-24891.
2018-07-24 19:35:34 -07:00
s71955 d4a277f0ce [SPARK-24812][SQL] Last Access Time in the table description is not valid
## What changes were proposed in this pull request?

Last Access Time will always displayed wrong date Thu Jan 01 05:30:00 IST 1970 when user run  DESC FORMATTED table command
In hive its displayed as "UNKNOWN" which makes more sense than displaying wrong date. seems to be a limitation as of now even from hive, better we can follow the hive behavior unless the limitation has been resolved from hive.

spark client output
![spark_desc table](https://user-images.githubusercontent.com/12999161/42753448-ddeea66a-88a5-11e8-94aa-ef8d017f94c5.png)

Hive client output
![hive_behaviour](https://user-images.githubusercontent.com/12999161/42753489-f4fd366e-88a5-11e8-83b0-0f3a53ce83dd.png)

## How was this patch tested?
UT has been added which makes sure that the wrong date "Thu Jan 01 05:30:00 IST 1970 "
shall not be added as value for the Last Access  property

Author: s71955 <sujithchacko.2010@gmail.com>

Closes #21775 from sujith71955/master_hive.
2018-07-24 11:31:27 -07:00
hyukjinkwon 3d5c61e5fd [SPARK-22499][FOLLOWUP][SQL] Reduce input string expressions for Least and Greatest to reduce time in its test
## What changes were proposed in this pull request?

It's minor and trivial but looks 2000 input is good enough to reproduce and test in SPARK-22499.

## How was this patch tested?

Manually brought the change and tested.

Locally tested:

Before: 3m 21s 288ms
After: 1m 29s 134ms

Given the latest successful build took:

```
ArithmeticExpressionSuite:
- SPARK-22499: Least and greatest should not generate codes beyond 64KB (7 minutes, 49 seconds)
```

I expect it's going to save 4ish mins.

Author: hyukjinkwon <gurwls223@apache.org>

Closes #21855 from HyukjinKwon/minor-fix-suite.
2018-07-24 19:51:09 +08:00
10129659 13a67b070d [SPARK-24870][SQL] Cache can't work normally if there are case letters in SQL
## What changes were proposed in this pull request?
Modified the canonicalized to not case-insensitive.
Before the PR, cache can't work normally if there are case letters in SQL,
for example:
     sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING) USING hive")

    sql("select key, sum(case when Key > 0 then 1 else 0 end) as positiveNum " +
      "from src group by key").cache().createOrReplaceTempView("src_cache")
    sql(
      s"""select a.key
           from
           (select key from src_cache where positiveNum = 1)a
           left join
           (select key from src_cache )b
           on a.key=b.key
        """).explain

The physical plan of the sql is:
![image](https://user-images.githubusercontent.com/26834091/42979518-3decf0fa-8c05-11e8-9837-d5e4c334cb1f.png)

The subquery "select key from src_cache where positiveNum = 1" on the left of join can use the cache data, but the subquery "select key from src_cache" on the right of join cannot use the cache data.

## How was this patch tested?

new added test

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

Closes #21823 from eatoncys/canonicalized.
2018-07-23 23:05:08 -07:00
Yuanjian Li cfc3e1aaa4 [SPARK-24339][SQL] Prunes the unused columns from child of ScriptTransformation
## What changes were proposed in this pull request?

Modify the strategy in ColumnPruning to add a Project between ScriptTransformation and its child, this strategy can reduce the scan time especially in the scenario of the table has many columns.

## How was this patch tested?

Add UT in ColumnPruningSuite and ScriptTransformationSuite.

Author: Yuanjian Li <xyliyuanjian@gmail.com>

Closes #21839 from xuanyuanking/SPARK-24339.
2018-07-23 13:04:39 -07:00
maryannxue 434319e73f [SPARK-24802][SQL] Add a new config for Optimization Rule Exclusion
## What changes were proposed in this pull request?

Since Spark has provided fairly clear interfaces for adding user-defined optimization rules, it would be nice to have an easy-to-use interface for excluding an optimization rule from the Spark query optimizer as well.

This would make customizing Spark optimizer easier and sometimes could debugging issues too.

- Add a new config spark.sql.optimizer.excludedRules, with the value being a list of rule names separated by comma.
- Modify the current batches method to remove the excluded rules from the default batches. Log the rules that have been excluded.
- Split the existing default batches into "post-analysis batches" and "optimization batches" so that only rules in the "optimization batches" can be excluded.

## How was this patch tested?

Add a new test suite: OptimizerRuleExclusionSuite

Author: maryannxue <maryannxue@apache.org>

Closes #21764 from maryannxue/rule-exclusion.
2018-07-23 08:25:24 -07:00
Gengliang Wang 8817c68f50 [SPARK-24811][SQL] Avro: add new function from_avro and to_avro
## What changes were proposed in this pull request?

1. Add a new function from_avro for parsing a binary column of avro format and converting it into its corresponding catalyst value.

2. Add a new function to_avro for converting a column into binary of avro format with the specified schema.

I created #21774 for this, but it failed the build https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Compile/job/spark-master-compile-maven-hadoop-2.6/7902/

Additional changes In this PR:
1. Add `scalacheck` dependency in pom.xml to resolve the failure.
2. Update the `log4j.properties` to make it consistent with other modules.

## How was this patch tested?

Unit test
Compile with different commands:
```
./build/mvn --force -DzincPort=3643 -DskipTests -Phadoop-2.6 -Phive-thriftserver -Pkinesis-asl -Pspark-ganglia-lgpl -Pmesos -Pyarn  compile test-compile
./build/mvn --force -DzincPort=3643 -DskipTests -Phadoop-2.7 -Phive-thriftserver -Pkinesis-asl -Pspark-ganglia-lgpl -Pmesos -Pyarn  compile test-compile
./build/mvn --force -DzincPort=3643 -DskipTests -Phadoop-3.1 -Phive-thriftserver -Pkinesis-asl -Pspark-ganglia-lgpl -Pmesos -Pyarn  compile test-compile
```

Author: Gengliang Wang <gengliang.wang@databricks.com>

Closes #21838 from gengliangwang/from_and_to_avro.
2018-07-22 17:36:57 -07:00
Brandon Krieger 597bdeff2d [SPARK-24488][SQL] Fix issue when generator is aliased multiple times
## What changes were proposed in this pull request?

Currently, the Analyzer throws an exception if your try to nest a generator. However, it special cases generators "nested" in an alias, and allows that. If you try to alias a generator twice, it is not caught by the special case, so an exception is thrown.

This PR trims the unnecessary, non-top-level aliases, so that the generator is allowed.

## How was this patch tested?

new tests in AnalysisSuite.

Author: Brandon Krieger <bkrieger@palantir.com>

Closes #21508 from bkrieger/bk/SPARK-24488.
2018-07-21 00:44:00 +02:00
Xiao Li 9ad77b3037 Revert "[SPARK-24811][SQL] Avro: add new function from_avro and to_avro"
This reverts commit 244bcff194.
2018-07-20 12:55:38 -07:00
Gengliang Wang 244bcff194 [SPARK-24811][SQL] Avro: add new function from_avro and to_avro
## What changes were proposed in this pull request?

Add a new function from_avro for parsing a binary column of avro format and converting it into its corresponding catalyst value.

Add a new function to_avro for converting a column into binary of avro format with the specified schema.

This PR is in progress. Will add test cases.
## How was this patch tested?

Author: Gengliang Wang <gengliang.wang@databricks.com>

Closes #21774 from gengliangwang/from_and_to_avro.
2018-07-20 09:19:29 -07:00
Takuya UESHIN 7b6d36bc9e [SPARK-24871][SQL] Refactor Concat and MapConcat to avoid creating concatenator object for each row.
## What changes were proposed in this pull request?

Refactor `Concat` and `MapConcat` to:

- avoid creating concatenator object for each row.
- make `Concat` handle `containsNull` properly.
- make `Concat` shortcut if `null` child is found.

## How was this patch tested?

Added some tests and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #21824 from ueshin/issues/SPARK-24871/refactor_concat_mapconcat.
2018-07-20 20:08:42 +08:00
Dilip Biswal 2b91d9918c [SPARK-24424][SQL] Support ANSI-SQL compliant syntax for GROUPING SET
## What changes were proposed in this pull request?

Enhances the parser and analyzer to support ANSI compliant syntax for GROUPING SET. As part of this change we derive the grouping expressions from user supplied groupings in the grouping sets clause.

```SQL
SELECT c1, c2, max(c3)
FROM t1
GROUP BY GROUPING SETS ((c1), (c1, c2))
```

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

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

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

Closes #21813 from dilipbiswal/spark-24424.
2018-07-19 23:52:53 -07:00
Marco Gaido a5925c1631 [SPARK-24268][SQL] Use datatype.catalogString in error messages
## What changes were proposed in this pull request?

As stated in https://github.com/apache/spark/pull/21321, in the error messages we should use `catalogString`. This is not the case, as SPARK-22893 used `simpleString` in order to have the same representation everywhere and it missed some places.

The PR unifies the messages using alway the `catalogString` representation of the dataTypes in the messages.

## How was this patch tested?

existing/modified UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21804 from mgaido91/SPARK-24268_catalog.
2018-07-19 23:29:29 -07:00
Ger van Rossum 67e108daa6 [SPARK-24846][SQL] Made hashCode ExprId independent of jvmId
## What changes were proposed in this pull request?
Made ExprId hashCode independent of jvmId to make canonicalization independent of JVM, by overriding hashCode (and necessarily also equality) to depend on id only

## How was this patch tested?
Created a unit test ExprIdSuite
Ran all unit tests of sql/catalyst

Author: Ger van Rossum <gvr@users.noreply.github.com>

Closes #21806 from gvr/spark24846-canonicalization.
2018-07-19 23:28:16 +02:00
Tathagata Das b3d88ac029 [SPARK-22187][SS] Update unsaferow format for saved state in flatMapGroupsWithState to allow timeouts with deleted state
## What changes were proposed in this pull request?

Currently, the group state of user-defined-type is encoded as top-level columns in the UnsafeRows stores in the state store. The timeout timestamp is also saved as (when needed) as the last top-level column. Since the group state is serialized to top-level columns, you cannot save "null" as a value of state (setting null in all the top-level columns is not equivalent). So we don't let the user set the timeout without initializing the state for a key. Based on user experience, this leads to confusion.

This PR is to change the row format such that the state is saved as nested columns. This would allow the state to be set to null, and avoid these confusing corner cases. However, queries recovering from existing checkpoint will use the previous format to maintain compatibility with existing production queries.

## How was this patch tested?
Refactored existing end-to-end tests and added new tests for explicitly testing obj-to-row conversion for both state formats.

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

Closes #21739 from tdas/SPARK-22187-1.
2018-07-19 13:17:28 -07:00
Jungtaek Lim 8b7d4f842f [SPARK-24717][SS] Split out max retain version of state for memory in HDFSBackedStateStoreProvider
## What changes were proposed in this pull request?

This patch proposes breaking down configuration of retaining batch size on state into two pieces: files and in memory (cache). While this patch reuses existing configuration for files, it introduces new configuration, "spark.sql.streaming.maxBatchesToRetainInMemory" to configure max count of batch to retain in memory.

## How was this patch tested?

Apply this patch on top of SPARK-24441 (https://github.com/apache/spark/pull/21469), and manually tested in various workloads to ensure overall size of states in memory is around 2x or less of the size of latest version of state, while it was 10x ~ 80x before applying the patch.

Author: Jungtaek Lim <kabhwan@gmail.com>

Closes #21700 from HeartSaVioR/SPARK-24717.
2018-07-19 00:07:35 -07:00
Sean Owen 753f115162 [SPARK-21261][DOCS][SQL] SQL Regex document fix
## What changes were proposed in this pull request?

Fix regexes in spark-sql command examples.
This takes over https://github.com/apache/spark/pull/18477

## How was this patch tested?

Existing tests. I verified the existing example doesn't work in spark-sql, but new ones does.

Author: Sean Owen <srowen@gmail.com>

Closes #21808 from srowen/SPARK-21261.
2018-07-18 18:39:23 -05:00
maryannxue cd203e0dfc [SPARK-24163][SPARK-24164][SQL] Support column list as the pivot column in Pivot
## What changes were proposed in this pull request?

1. Extend the Parser to enable parsing a column list as the pivot column.
2. Extend the Parser and the Pivot node to enable parsing complex expressions with aliases as the pivot value.
3. Add type check and constant check in Analyzer for Pivot node.

## How was this patch tested?

Add tests in pivot.sql

Author: maryannxue <maryannxue@apache.org>

Closes #21720 from maryannxue/spark-24164.
2018-07-18 13:33:26 -07:00
DB Tsai 681845fd62
[SPARK-24402][SQL] Optimize In expression when only one element in the collection or collection is empty
## What changes were proposed in this pull request?

Two new rules in the logical plan optimizers are added.

1. When there is only one element in the **`Collection`**, the
physical plan will be optimized to **`EqualTo`**, so predicate
pushdown can be used.

```scala
    profileDF.filter( $"profileID".isInCollection(Set(6))).explain(true)
    """
      |== Physical Plan ==
      |*(1) Project [profileID#0]
      |+- *(1) Filter (isnotnull(profileID#0) && (profileID#0 = 6))
      |   +- *(1) FileScan parquet [profileID#0] Batched: true, Format: Parquet,
      |     PartitionFilters: [],
      |     PushedFilters: [IsNotNull(profileID), EqualTo(profileID,6)],
      |     ReadSchema: struct<profileID:int>
    """.stripMargin
```

2. When the **`Collection`** is empty, and the input is nullable, the
logical plan will be simplified to

```scala
    profileDF.filter( $"profileID".isInCollection(Set())).explain(true)
    """
      |== Optimized Logical Plan ==
      |Filter if (isnull(profileID#0)) null else false
      |+- Relation[profileID#0] parquet
    """.stripMargin
```

TODO:

1. For multiple conditions with numbers less than certain thresholds,
we should still allow predicate pushdown.
2. Optimize the **`In`** using **`tableswitch`** or **`lookupswitch`**
when the numbers of the categories are low, and they are **`Int`**,
**`Long`**.
3. The default immutable hash trees set is slow for query, and we
should do benchmark for using different set implementation for faster
query.
4. **`filter(if (condition) null else false)`** can be optimized to false.

## How was this patch tested?

Couple new tests are added.

Author: DB Tsai <d_tsai@apple.com>

Closes #21797 from dbtsai/optimize-in.
2018-07-17 17:33:52 -07:00
HanShuliang 7688ce88b2 [SPARK-21590][SS] Window start time should support negative values
## What changes were proposed in this pull request?

Remove the non-negative checks of window start time to make window support negative start time, and add a check to guarantee the absolute value of start time is less than slide duration.

## How was this patch tested?

New unit tests.

Author: HanShuliang <kevinzwx1992@gmail.com>

Closes #18903 from KevinZwx/dev.
2018-07-17 11:25:23 -05:00
Marek Novotny 4cf1bec4dc [SPARK-24305][SQL][FOLLOWUP] Avoid serialization of private fields in collection expressions.
## What changes were proposed in this pull request?

The PR tries to avoid serialization of private fields of already added collection functions and follows up on comments in [SPARK-23922](https://github.com/apache/spark/pull/21028) and [SPARK-23935](https://github.com/apache/spark/pull/21236)

## How was this patch tested?

Run tests from:
- CollectionExpressionSuite.scala
- DataFrameFunctionsSuite.scala

Author: Marek Novotny <mn.mikke@gmail.com>

Closes #21352 from mn-mikke/SPARK-24305.
2018-07-17 23:07:18 +08:00
hyukjinkwon 0ca16f6e14 Revert "[SPARK-24402][SQL] Optimize In expression when only one element in the collection or collection is empty"
This reverts commit 0f0d1865f5.
2018-07-17 11:30:53 +08:00
DB Tsai 0f0d1865f5 [SPARK-24402][SQL] Optimize In expression when only one element in the collection or collection is empty
## What changes were proposed in this pull request?

Two new rules in the logical plan optimizers are added.

1. When there is only one element in the **`Collection`**, the
physical plan will be optimized to **`EqualTo`**, so predicate
pushdown can be used.

```scala
    profileDF.filter( $"profileID".isInCollection(Set(6))).explain(true)
    """
      |== Physical Plan ==
      |*(1) Project [profileID#0]
      |+- *(1) Filter (isnotnull(profileID#0) && (profileID#0 = 6))
      |   +- *(1) FileScan parquet [profileID#0] Batched: true, Format: Parquet,
      |     PartitionFilters: [],
      |     PushedFilters: [IsNotNull(profileID), EqualTo(profileID,6)],
      |     ReadSchema: struct<profileID:int>
    """.stripMargin
```

2. When the **`Collection`** is empty, and the input is nullable, the
logical plan will be simplified to

```scala
    profileDF.filter( $"profileID".isInCollection(Set())).explain(true)
    """
      |== Optimized Logical Plan ==
      |Filter if (isnull(profileID#0)) null else false
      |+- Relation[profileID#0] parquet
    """.stripMargin
```

TODO:

1. For multiple conditions with numbers less than certain thresholds,
we should still allow predicate pushdown.
2. Optimize the **`In`** using **`tableswitch`** or **`lookupswitch`**
when the numbers of the categories are low, and they are **`Int`**,
**`Long`**.
3. The default immutable hash trees set is slow for query, and we
should do benchmark for using different set implementation for faster
query.
4. **`filter(if (condition) null else false)`** can be optimized to false.

## How was this patch tested?

Couple new tests are added.

Author: DB Tsai <d_tsai@apple.com>

Closes #21442 from dbtsai/optimize-in.
2018-07-16 15:33:39 -07:00
Marek Novotny b0c95a1d69 [SPARK-23901][SQL] Removing masking functions
The PR reverts #21246.

Author: Marek Novotny <mn.mikke@gmail.com>

Closes #21786 from mn-mikke/SPARK-23901.
2018-07-16 14:28:35 -07:00
Takuya UESHIN b045315e5d [SPARK-24734][SQL] Fix type coercions and nullabilities of nested data types of some functions.
## What changes were proposed in this pull request?

We have some functions which need to aware the nullabilities of all children, such as `CreateArray`, `CreateMap`, `Concat`, and so on. Currently we add casts to fix the nullabilities, but the casts might be removed during the optimization phase.
After the discussion, we decided to not add extra casts for just fixing the nullabilities of the nested types, but handle them by functions themselves.

## How was this patch tested?

Modified and added some tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #21704 from ueshin/issues/SPARK-24734/concat_containsnull.
2018-07-16 23:16:25 +08:00
Yuming Wang 9549a28149 [SPARK-24549][SQL] Support Decimal type push down to the parquet data sources
## What changes were proposed in this pull request?

Support Decimal type push down to the parquet data sources.
The Decimal comparator used is: [`BINARY_AS_SIGNED_INTEGER_COMPARATOR`](c6764c4a08/parquet-column/src/main/java/org/apache/parquet/schema/PrimitiveComparator.java (L224-L292)).

## How was this patch tested?

unit tests and manual tests.

**manual tests**:
```scala
spark.range(10000000).selectExpr("id", "cast(id as decimal(9)) as d1", "cast(id as decimal(9, 2)) as d2", "cast(id as decimal(18)) as d3", "cast(id as decimal(18, 4)) as d4", "cast(id as decimal(38)) as d5", "cast(id as decimal(38, 18)) as d6").coalesce(1).write.option("parquet.block.size", 1048576).parquet("/tmp/spark/parquet/decimal")
val df = spark.read.parquet("/tmp/spark/parquet/decimal/")
spark.sql("set spark.sql.parquet.filterPushdown.decimal=true")
// Only read about 1 MB data
df.filter("d2 = 10000").show
// Only read about 1 MB data
df.filter("d4 = 10000").show
spark.sql("set spark.sql.parquet.filterPushdown.decimal=false")
// Read 174.3 MB data
df.filter("d2 = 10000").show
// Read 174.3 MB data
df.filter("d4 = 10000").show
```

Author: Yuming Wang <yumwang@ebay.com>

Closes #21556 from wangyum/SPARK-24549.
2018-07-16 15:44:51 +08:00
Yuming Wang 43e4e851b6 [SPARK-24718][SQL] Timestamp support pushdown to parquet data source
## What changes were proposed in this pull request?

`Timestamp` support pushdown to parquet data source.
Only `TIMESTAMP_MICROS` and `TIMESTAMP_MILLIS` support push down.

## How was this patch tested?

unit tests and benchmark tests

Author: Yuming Wang <yumwang@ebay.com>

Closes #21741 from wangyum/SPARK-24718.
2018-07-15 11:13:49 +08:00
Yuming Wang e1de34113e [SPARK-17091][SQL] Add rule to convert IN predicate to equivalent Parquet filter
## What changes were proposed in this pull request?

The original pr is: https://github.com/apache/spark/pull/18424

Add a new optimizer rule to convert an IN predicate to an equivalent Parquet filter and add `spark.sql.parquet.pushdown.inFilterThreshold` to control limit thresholds. Different data types have different limit thresholds, this is a copy of data for reference:

Type | limit threshold
-- | --
string | 370
int | 210
long | 285
double | 270
float | 220
decimal | Won't provide better performance before [SPARK-24549](https://issues.apache.org/jira/browse/SPARK-24549)

## How was this patch tested?
unit tests and manual tests

Author: Yuming Wang <yumwang@ebay.com>

Closes #21603 from wangyum/SPARK-17091.
2018-07-14 17:50:54 +08:00
Liang-Chi Hsieh dfd7ac9887 [SPARK-24781][SQL] Using a reference from Dataset in Filter/Sort might not work
## What changes were proposed in this pull request?

When we use a reference from Dataset in filter or sort, which was not used in the prior select, an AnalysisException occurs, e.g.,

```scala
val df = Seq(("test1", 0), ("test2", 1)).toDF("name", "id")
df.select(df("name")).filter(df("id") === 0).show()
```

```scala
org.apache.spark.sql.AnalysisException: Resolved attribute(s) id#6 missing from name#5 in operator !Filter (id#6 = 0).;;
!Filter (id#6 = 0)
   +- AnalysisBarrier
      +- Project [name#5]
         +- Project [_1#2 AS name#5, _2#3 AS id#6]
            +- LocalRelation [_1#2, _2#3]
```
This change updates the rule `ResolveMissingReferences` so `Filter` and `Sort` with non-empty `missingInputs` will also be transformed.

## How was this patch tested?

Added tests.

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

Closes #21745 from viirya/SPARK-24781.
2018-07-13 08:25:00 -07:00
Kevin Yu 0ce11d0e3a [SPARK-23486] cache the function name from the external catalog for lookupFunctions
## What changes were proposed in this pull request?

This PR will cache the function name from external catalog, it is used by lookupFunctions in the analyzer, and it is cached for each query plan. The original problem is reported in the [ spark-19737](https://issues.apache.org/jira/browse/SPARK-19737)

## How was this patch tested?

create new test file LookupFunctionsSuite and add test case in SessionCatalogSuite

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

Closes #20795 from kevinyu98/spark-23486.
2018-07-12 22:20:06 -07:00
maryannxue 75725057b3 [SPARK-24790][SQL] Allow complex aggregate expressions in Pivot
## What changes were proposed in this pull request?

Relax the check to allow complex aggregate expressions, like `ceil(sum(col1))` or `sum(col1) + 1`, which roughly means any aggregate expression that could appear in an Aggregate plan except pandas UDF (due to the fact that it is not supported in pivot yet).

## How was this patch tested?

Added 2 tests in pivot.sql

Author: maryannxue <maryannxue@apache.org>

Closes #21753 from maryannxue/pivot-relax-syntax.
2018-07-12 16:54:03 -07:00
Marco Gaido 11384893b6 [SPARK-24208][SQL][FOLLOWUP] Move test cases to proper locations
## What changes were proposed in this pull request?

The PR is a followup to move the test cases introduced by the original PR in their proper location.

## How was this patch tested?

moved UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21751 from mgaido91/SPARK-24208_followup.
2018-07-12 15:13:26 -07:00
Kazuaki Ishizaki 301bff7063 [SPARK-23914][SQL] Add array_union function
## What changes were proposed in this pull request?

The PR adds the SQL function `array_union`. The behavior of the function is based on Presto's one.

This function returns returns an array of the elements in the union of array1 and array2.

Note: The order of elements in the result is not defined.

## How was this patch tested?

Added UTs

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

Closes #21061 from kiszk/SPARK-23914.
2018-07-12 17:42:29 +09:00
Kazuaki Ishizaki 5ad4735bda [SPARK-24529][BUILD][TEST-MAVEN] Add spotbugs into maven build process
## What changes were proposed in this pull request?

This PR enables a Java bytecode check tool [spotbugs](https://spotbugs.github.io/) to avoid possible integer overflow at multiplication. When an violation is detected, the build process is stopped.
Due to the tool limitation, some other checks will be enabled. In this PR, [these patterns](http://spotbugs-in-kengo-toda.readthedocs.io/en/lqc-list-detectors/detectors.html#findpuzzlers) in `FindPuzzlers` can be detected.

This check is enabled at `compile` phase. Thus, `mvn compile` or `mvn package` launches this check.

## How was this patch tested?

Existing UTs

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

Closes #21542 from kiszk/SPARK-24529.
2018-07-12 09:52:23 +08:00
Maxim Gekk 3ab48f985c [SPARK-24761][SQL] Adding of isModifiable() to RuntimeConfig
## What changes were proposed in this pull request?

In the PR, I propose to extend `RuntimeConfig` by new method `isModifiable()` which returns `true` if a config parameter can be modified at runtime (for current session state). For static SQL and core parameters, the method returns `false`.

## How was this patch tested?

Added new test to `RuntimeConfigSuite` for checking Spark core and SQL parameters.

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21730 from MaxGekk/is-modifiable.
2018-07-11 17:38:43 -07:00
Marco Gaido e008ad1752 [SPARK-24782][SQL] Simplify conf retrieval in SQL expressions
## What changes were proposed in this pull request?

The PR simplifies the retrieval of config in `size`, as we can access them from tasks too thanks to SPARK-24250.

## How was this patch tested?

existing UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21736 from mgaido91/SPARK-24605_followup.
2018-07-11 17:30:43 -07:00
Marco Gaido ebf4bfb966 [SPARK-24208][SQL] Fix attribute deduplication for FlatMapGroupsInPandas
## What changes were proposed in this pull request?

A self-join on a dataset which contains a `FlatMapGroupsInPandas` fails because of duplicate attributes. This happens because we are not dealing with this specific case in our `dedupAttr` rules.

The PR fix the issue by adding the management of the specific case

## How was this patch tested?

added UT + manual tests

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

Closes #21737 from mgaido91/SPARK-24208.
2018-07-11 09:29:19 -07:00
Marek Novotny 74a8d6308b [SPARK-24165][SQL] Fixing conditional expressions to handle nullability of nested types
## What changes were proposed in this pull request?
This PR is proposing a fix for the output data type of ```If``` and ```CaseWhen``` expression. Upon till now, the implementation of exprassions has ignored nullability of nested types from different execution branches and returned the type of the first branch.

This could lead to an unwanted ```NullPointerException``` from other expressions depending on a ```If```/```CaseWhen``` expression.

Example:
```
val rows = new util.ArrayList[Row]()
rows.add(Row(true, ("a", 1)))
rows.add(Row(false, (null, 2)))
val schema = StructType(Seq(
  StructField("cond", BooleanType, false),
  StructField("s", StructType(Seq(
    StructField("val1", StringType, true),
    StructField("val2", IntegerType, false)
  )), false)
))

val df = spark.createDataFrame(rows, schema)

df
  .select(when('cond, struct(lit("x").as("val1"), lit(10).as("val2"))).otherwise('s) as "res")
  .select('res.getField("val1"))
  .show()
```
Exception:
```
Exception in thread "main" java.lang.NullPointerException
	at org.apache.spark.sql.catalyst.expressions.codegen.UnsafeWriter.write(UnsafeWriter.java:109)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source)
	at org.apache.spark.sql.execution.LocalTableScanExec$$anonfun$unsafeRows$1.apply(LocalTableScanExec.scala:44)
	at org.apache.spark.sql.execution.LocalTableScanExec$$anonfun$unsafeRows$1.apply(LocalTableScanExec.scala:44)
...
```
Output schema:
```
root
 |-- res.val1: string (nullable = false)
```

## How was this patch tested?
New test cases added into
- DataFrameSuite.scala
- conditionalExpressions.scala

Author: Marek Novotny <mn.mikke@gmail.com>

Closes #21687 from mn-mikke/SPARK-24165.
2018-07-11 12:21:03 +08:00
Tathagata Das 6078b891da [SPARK-24730][SS] Add policy to choose max as global watermark when streaming query has multiple watermarks
## What changes were proposed in this pull request?

Currently, when a streaming query has multiple watermark, the policy is to choose the min of them as the global watermark. This is safe to do as the global watermark moves with the slowest stream, and is therefore is safe as it does not unexpectedly drop some data as late, etc. While this is indeed the safe thing to do, in some cases, you may want the watermark to advance with the fastest stream, that is, take the max of multiple watermarks. This PR is to add that configuration. It makes the following changes.

- Adds a configuration to specify max as the policy.
- Saves the configuration in OffsetSeqMetadata because changing it in the middle can lead to unpredictable results.
   - For old checkpoints without the configuration, it assumes the default policy as min (irrespective of the policy set at the session where the query is being restarted). This is to ensure that existing queries are affected in any way.

TODO
- [ ] Add a test for recovery from existing checkpoints.

## How was this patch tested?
New unit test

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

Closes #21701 from tdas/SPARK-24730.
2018-07-10 18:03:40 -07:00
Mukul Murthy 32cb50835e [SPARK-24662][SQL][SS] Support limit in structured streaming
## What changes were proposed in this pull request?

Support the LIMIT operator in structured streaming.

For streams in append or complete output mode, a stream with a LIMIT operator will return no more than the specified number of rows. LIMIT is still unsupported for the update output mode.

This change reverts e4fee395ec as part of it because it is a better and more complete implementation.

## How was this patch tested?

New and existing unit tests.

Author: Mukul Murthy <mukul.murthy@gmail.com>

Closes #21662 from mukulmurthy/SPARK-24662.
2018-07-10 11:08:04 -07:00
Xiao Li aec966b05e Revert "[SPARK-24268][SQL] Use datatype.simpleString in error messages"
This reverts commit 1bd3d61f41.
2018-07-09 14:24:23 -07:00
Marco Gaido 1bd3d61f41 [SPARK-24268][SQL] Use datatype.simpleString in error messages
## What changes were proposed in this pull request?

SPARK-22893 tried to unify error messages about dataTypes. Unfortunately, still many places were missing the `simpleString` method in other to have the same representation everywhere.

The PR unified the messages using alway the simpleString representation of the dataTypes in the messages.

## How was this patch tested?

existing/modified UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21321 from mgaido91/SPARK-24268.
2018-07-09 22:59:05 +08:00
Bruce Robbins 034913b62b [SPARK-23936][SQL] Implement map_concat
## What changes were proposed in this pull request?

Implement map_concat high order function.

This implementation does not pick a winner when the specified maps have overlapping keys. Therefore, this implementation preserves existing duplicate keys in the maps and potentially introduces new duplicates (After discussion with ueshin, we settled on option 1 from [here](https://issues.apache.org/jira/browse/SPARK-23936?focusedCommentId=16464245&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16464245)).

## How was this patch tested?

New tests
Manual tests
Run all sbt SQL tests
Run all pyspark sql tests

Author: Bruce Robbins <bersprockets@gmail.com>

Closes #21073 from bersprockets/SPARK-23936.
2018-07-09 21:21:38 +09:00
Liang-Chi Hsieh fc43690d36 [SPARK-24749][SQL] Use sameType to compare Array's element type in ArrayContains
## What changes were proposed in this pull request?

We should use `DataType.sameType` to compare element type in `ArrayContains`, otherwise nullability affects comparison result.

## How was this patch tested?

Added test.

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

Closes #21724 from viirya/SPARK-24749.
2018-07-07 11:34:30 +08:00
Liang-Chi Hsieh 4de0425df8 [SPARK-24569][SQL] Aggregator with output type Option should produce consistent schema
## What changes were proposed in this pull request?

SQL `Aggregator` with output type `Option[Boolean]` creates column of type `StructType`. It's not in consistency with a Dataset of similar java class.

This changes the way `definedByConstructorParams` checks given type. For `Option[_]`, it goes to check its type argument.

## How was this patch tested?

Added test.

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

Closes #21611 from viirya/SPARK-24569.
2018-07-07 10:54:14 +08:00
Takuya UESHIN 01fcba2c68 [SPARK-24737][SQL] Type coercion between StructTypes.
## What changes were proposed in this pull request?

We can support type coercion between `StructType`s where all the internal types are compatible.

## How was this patch tested?

Added tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #21713 from ueshin/issues/SPARK-24737/structtypecoercion.
2018-07-06 11:10:50 +08:00
Gengliang Wang 33952cfa81 [SPARK-24675][SQL] Rename table: validate existence of new location
## What changes were proposed in this pull request?
If table is renamed to a existing new location, data won't show up.
```
scala>  Seq("hello").toDF("a").write.format("parquet").saveAsTable("t")

scala> sql("select * from t").show()
+-----+
|    a|
+-----+
|hello|
+-----+

scala> sql("alter table t rename to test")
res2: org.apache.spark.sql.DataFrame = []

scala> sql("select * from test").show()
+---+
|  a|
+---+
+---+
```
The file layout is like
```
$ tree test
test
├── gabage
└── t
    ├── _SUCCESS
    └── part-00000-856b0f10-08f1-42d6-9eb3-7719261f3d5e-c000.snappy.parquet
```

In Hive, if the new location exists, the renaming will fail even the location is empty.

We should have the same validation in Catalog, in case of unexpected bugs.

## How was this patch tested?

New unit test.

Author: Gengliang Wang <gengliang.wang@databricks.com>

Closes #21655 from gengliangwang/validate_rename_table.
2018-07-05 09:25:19 -07:00
Liang-Chi Hsieh 32cfd3e75a [SPARK-24361][SQL] Polish code block manipulation API
## What changes were proposed in this pull request?

Current code block manipulation API is immature and hacky. We need a formal API to manipulate code blocks.

The basic idea is making `JavaCode`  as `TreeNode`. So we can use familiar `transform` API to manipulate code blocks and expressions in code blocks.

For example, we can replace `SimpleExprValue` in a code block like this:

```scala
code.transformExprValues {
  case SimpleExprValue("1 + 1", _) => aliasedParam
}
```

The example use case is splitting code to methods.

For example, we have an `ExprCode` containing generated code. But it is too long and we need to split it as method. Because statement-based expressions can't be directly passed into. We need to transform them as variables first:

```scala

def getExprValues(block: Block): Set[ExprValue] = block match {
  case c: CodeBlock =>
    c.blockInputs.collect {
      case e: ExprValue => e
    }.toSet
  case _ => Set.empty
}

def currentCodegenInputs(ctx: CodegenContext): Set[ExprValue] = {
  // Collects current variables in ctx.currentVars and ctx.INPUT_ROW.
  // It looks roughly like...
  ctx.currentVars.flatMap { v =>
    getExprValues(v.code) ++ Set(v.value, v.isNull)
  }.toSet + ctx.INPUT_ROW
}

// A code block of an expression contains too long code, making it as method
if (eval.code.length > 1024) {
  val setIsNull = if (!eval.isNull.isInstanceOf[LiteralValue]) {
    ...
  } else {
    ""
  }

  // Pick up variables and statements necessary to pass in.
  val currentVars = currentCodegenInputs(ctx)
  val varsPassIn = getExprValues(eval.code).intersect(currentVars)
  val aliasedExprs = HashMap.empty[SimpleExprValue, VariableValue]

  // Replace statement-based expressions which can't be directly passed in the method.
  val newCode = eval.code.transform {
    case block =>
      block.transformExprValues {
        case s: SimpleExprValue(_, javaType) if varsPassIn.contains(s) =>
          if (aliasedExprs.contains(s)) {
            aliasedExprs(s)
          } else {
            val aliasedVariable = JavaCode.variable(ctx.freshName("aliasedVar"), javaType)
            aliasedExprs += s -> aliasedVariable
            varsPassIn += aliasedVariable
            aliasedVariable
          }
      }
  }

  val params = varsPassIn.filter(!_.isInstanceOf[SimpleExprValue])).map { variable =>
    s"${variable.javaType.getName} ${variable.variableName}"
  }.mkString(", ")

  val funcName = ctx.freshName("nodeName")
  val javaType = CodeGenerator.javaType(dataType)
  val newValue = JavaCode.variable(ctx.freshName("value"), dataType)
  val funcFullName = ctx.addNewFunction(funcName,
    s"""
      |private $javaType $funcName($params) {
      |  $newCode
      |  $setIsNull
      |  return ${eval.value};
      |}
    """.stripMargin))

  eval.value = newValue
  val args = varsPassIn.filter(!_.isInstanceOf[SimpleExprValue])).map { variable =>
    s"${variable.variableName}"
  }

  // Create a code block to assign statements to aliased variables.
  val createVariables = aliasedExprs.foldLeft(EmptyBlock) { (block, (statement, variable)) =>
    block + code"${statement.javaType.getName} $variable = $statement;"
  }
  eval.code = createVariables + code"$javaType $newValue = $funcFullName($args);"
}
```

## How was this patch tested?

Added unite tests.

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

Closes #21405 from viirya/codeblock-api.
2018-07-05 20:48:55 +08:00
Wenchen Fan bf764a33be [SPARK-22384][SQL][FOLLOWUP] Refine partition pruning when attribute is wrapped in Cast
## What changes were proposed in this pull request?

As mentioned in https://github.com/apache/spark/pull/21586 , `Cast.mayTruncate` is not 100% safe, string to boolean is allowed. Since changing `Cast.mayTruncate` also changes the behavior of Dataset, here I propose to add a new `Cast.canSafeCast` for partition pruning.

## How was this patch tested?

new test cases

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21712 from cloud-fan/safeCast.
2018-07-04 18:36:09 -07:00
Liang-Chi Hsieh 1a2655a9e7 [SPARK-24635][SQL] Remove Blocks class from JavaCode class hierarchy
## What changes were proposed in this pull request?

The `Blocks` class in `JavaCode` class hierarchy is not necessary. Its function can be taken by `CodeBlock`. We should remove it to make simpler class hierarchy.

## How was this patch tested?

Existing tests.

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

Closes #21619 from viirya/SPARK-24635.
2018-07-04 20:42:08 +08:00
Takeshi Yamamuro b2deef64f6 [SPARK-24727][SQL] Add a static config to control cache size for generated classes
## What changes were proposed in this pull request?
Since SPARK-24250 has been resolved, executors correctly references user-defined configurations. So, this pr added a static config to control cache size for generated classes in `CodeGenerator`.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21705 from maropu/SPARK-24727.
2018-07-04 20:04:18 +08:00
Takuya UESHIN 7c08eb6d61 [SPARK-24732][SQL] Type coercion between MapTypes.
## What changes were proposed in this pull request?

Currently we don't allow type coercion between maps.
We can support type coercion between MapTypes where both the key types and the value types are compatible.

## How was this patch tested?

Added tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #21703 from ueshin/issues/SPARK-24732/maptypecoercion.
2018-07-04 12:21:26 +08:00
Maxim Gekk 776f299fc8 [SPARK-24709][SQL] schema_of_json() - schema inference from an example
## What changes were proposed in this pull request?

In the PR, I propose to add new function - *schema_of_json()* which infers schema of JSON string literal. The result of the function is a string containing a schema in DDL format.

One of the use cases is using of *schema_of_json()* in the combination with *from_json()*. Currently, _from_json()_ requires a schema as a mandatory argument. The *schema_of_json()* function will allow to point out an JSON string as an example which has the same schema as the first argument of _from_json()_. For instance:

```sql
select from_json(json_column, schema_of_json('{"c1": [0], "c2": [{"c3":0}]}'))
from json_table;
```

## How was this patch tested?

Added new test to `JsonFunctionsSuite`, `JsonExpressionsSuite` and SQL tests to `json-functions.sql`

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21686 from MaxGekk/infer_schema_json.
2018-07-04 09:38:18 +08:00
Yuanjian Li 8f91c697e2 [SPARK-24665][PYSPARK] Use SQLConf in PySpark to manage all sql configs
## What changes were proposed in this pull request?

Use SQLConf for PySpark to manage all sql configs, drop all the hard code in config usage.

## How was this patch tested?

Existing UT.

Author: Yuanjian Li <xyliyuanjian@gmail.com>

Closes #21648 from xuanyuanking/SPARK-24665.
2018-07-02 14:35:37 +08:00
Xiao Li d54d8b8630 simplify rand in dsl/package.scala 2018-06-29 23:51:13 -07:00
maryannxue 797971ed42 [SPARK-24696][SQL] ColumnPruning rule fails to remove extra Project
## What changes were proposed in this pull request?

The ColumnPruning rule tries adding an extra Project if an input node produces fields more than needed, but as a post-processing step, it needs to remove the lower Project in the form of "Project - Filter - Project" otherwise it would conflict with PushPredicatesThroughProject and would thus cause a infinite optimization loop. The current post-processing method is defined as:
```
  private def removeProjectBeforeFilter(plan: LogicalPlan): LogicalPlan = plan transform {
    case p1  Project(_, f  Filter(_, p2  Project(_, child)))
      if p2.outputSet.subsetOf(child.outputSet) =>
      p1.copy(child = f.copy(child = child))
  }
```
This method works well when there is only one Filter but would not if there's two or more Filters. In this case, there is a deterministic filter and a non-deterministic filter so they stay as separate filter nodes and cannot be combined together.

An simplified illustration of the optimization process that forms the infinite loop is shown below (F1 stands for the 1st filter, F2 for the 2nd filter, P for project, S for scan of relation, PredicatePushDown as abbrev. of PushPredicatesThroughProject):
```
                             F1 - F2 - P - S
PredicatePushDown      =>    F1 - P - F2 - S
ColumnPruning          =>    F1 - P - F2 - P - S
                       =>    F1 - P - F2 - S        (Project removed)
PredicatePushDown      =>    P - F1 - F2 - S
ColumnPruning          =>    P - F1 - P - F2 - S
                       =>    P - F1 - P - F2 - P - S
                       =>    P - F1 - F2 - P - S    (only one Project removed)
RemoveRedundantProject =>    F1 - F2 - P - S        (goes back to the loop start)
```
So the problem is the ColumnPruning rule adds a Project under a Filter (and fails to remove it in the end), and that new Project triggers PushPredicateThroughProject. Once the filters have been push through the Project, a new Project will be added by the ColumnPruning rule and this goes on and on.
The fix should be when adding Projects, the rule applies top-down, but later when removing extra Projects, the process should go bottom-up to ensure all extra Projects can be matched.

## How was this patch tested?

Added a optimization rule test in ColumnPruningSuite; and a end-to-end test in SQLQuerySuite.

Author: maryannxue <maryannxue@apache.org>

Closes #21674 from maryannxue/spark-24696.
2018-06-29 23:46:12 -07:00
Yuming Wang 03545ce6de [SPARK-24638][SQL] StringStartsWith support push down
## What changes were proposed in this pull request?

`StringStartsWith` support push down. About 50% savings in compute time.

## How was this patch tested?
unit tests, manual tests and performance test:
```scala
cat <<EOF > SPARK-24638.scala
def benchmark(func: () => Unit): Long = {
  val start = System.currentTimeMillis()
  for(i <- 0 until 100) { func() }
  val end = System.currentTimeMillis()
  end - start
}
val path = "/tmp/spark/parquet/string/"
spark.range(10000000).selectExpr("concat(id, 'str', id) as id").coalesce(1).write.mode("overwrite").option("parquet.block.size", 1048576).parquet(path)
val df = spark.read.parquet(path)

spark.sql("set spark.sql.parquet.filterPushdown.string.startsWith=true")
val pushdownEnable = benchmark(() => df.where("id like '999998%'").count())

spark.sql("set spark.sql.parquet.filterPushdown.string.startsWith=false")
val pushdownDisable = benchmark(() => df.where("id like '999998%'").count())

val improvements = pushdownDisable - pushdownEnable
println(s"improvements: $improvements")
EOF

bin/spark-shell -i SPARK-24638.scala
```
result:
```scala
Loading SPARK-24638.scala...
benchmark: (func: () => Unit)Long
path: String = /tmp/spark/parquet/string/
df: org.apache.spark.sql.DataFrame = [id: string]
res1: org.apache.spark.sql.DataFrame = [key: string, value: string]
pushdownEnable: Long = 11608
res2: org.apache.spark.sql.DataFrame = [key: string, value: string]
pushdownDisable: Long = 31981
improvements: Long = 20373
```

Author: Yuming Wang <yumwang@ebay.com>

Closes #21623 from wangyum/SPARK-24638.
2018-06-30 13:58:50 +08:00
Jose Torres f6e6899a8b [SPARK-24386][SS] coalesce(1) aggregates in continuous processing
## What changes were proposed in this pull request?

Provide a continuous processing implementation of coalesce(1), as well as allowing aggregates on top of it.

The changes in ContinuousQueuedDataReader and such are to use split.index (the ID of the partition within the RDD currently being compute()d) rather than context.partitionId() (the partition ID of the scheduled task within the Spark job - that is, the post coalesce writer). In the absence of a narrow dependency, these values were previously always the same, so there was no need to distinguish.

## How was this patch tested?

new unit test

Author: Jose Torres <torres.joseph.f+github@gmail.com>

Closes #21560 from jose-torres/coalesce.
2018-06-28 16:25:40 -07:00
debugger87 c04cb2d1b7 [SPARK-21687][SQL] Spark SQL should set createTime for Hive partition
## What changes were proposed in this pull request?

Set createTime for every hive partition created in Spark SQL, which could be used to manage data lifecycle in Hive warehouse. We found  that almost every partition modified by spark sql has not been set createTime.

```
mysql> select * from partitions where create_time=0 limit 1\G;
*************************** 1. row ***************************
         PART_ID: 1028584
     CREATE_TIME: 0
LAST_ACCESS_TIME: 1502203611
       PART_NAME: date=20170130
           SD_ID: 1543605
          TBL_ID: 211605
  LINK_TARGET_ID: NULL
1 row in set (0.27 sec)
```

## How was this patch tested?
 N/A

Author: debugger87 <yangchaozhong.2009@gmail.com>
Author: Chaozhong Yang <yangchaozhong.2009@gmail.com>

Closes #18900 from debugger87/fix/set-create-time-for-hive-partition.
2018-06-27 11:34:28 -07:00
Yuanjian Li 6a0b77a55d [SPARK-24215][PYSPARK][FOLLOW UP] Implement eager evaluation for DataFrame APIs in PySpark
## What changes were proposed in this pull request?

Address comments in #21370 and add more test.

## How was this patch tested?

Enhance test in pyspark/sql/test.py and DataFrameSuite

Author: Yuanjian Li <xyliyuanjian@gmail.com>

Closes #21553 from xuanyuanking/SPARK-24215-follow.
2018-06-27 10:43:06 -07:00
Takuya UESHIN 9a76f23c6a [SPARK-23927][SQL][FOLLOW-UP] Fix a build failure.
## What changes were proposed in this pull request?

This pr is a follow-up pr of #21155.
The #21155 removed unnecessary import at that time, but the import became necessary in another pr.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #21646 from ueshin/issues/SPARK-23927/fup1.
2018-06-27 11:52:48 +08:00
Vayda, Oleksandr: IT (PRG) 2669b4de3b [SPARK-23927][SQL] Add "sequence" expression
## What changes were proposed in this pull request?
The PR adds the SQL function ```sequence```.
https://issues.apache.org/jira/browse/SPARK-23927

The behavior of the function is based on Presto's one.
Ref: https://prestodb.io/docs/current/functions/array.html

- ```sequence(start, stop) → array<bigint>```
Generate a sequence of integers from ```start``` to ```stop```, incrementing by ```1``` if ```start``` is less than or equal to ```stop```, otherwise ```-1```.
- ```sequence(start, stop, step) → array<bigint>```
Generate a sequence of integers from ```start``` to ```stop```, incrementing by ```step```.
- ```sequence(start_date, stop_date) → array<date>```
Generate a sequence of dates from ```start_date``` to ```stop_date```, incrementing by ```interval 1 day``` if ```start_date``` is less than or equal to ```stop_date```, otherwise ```- interval 1 day```.
- ```sequence(start_date, stop_date, step_interval) → array<date>```
Generate a sequence of dates from ```start_date``` to ```stop_date```, incrementing by ```step_interval```. The type of ```step_interval``` is ```CalendarInterval```.
- ```sequence(start_timestemp, stop_timestemp) → array<timestamp>```
Generate a sequence of timestamps from ```start_timestamps``` to ```stop_timestamps```, incrementing by ```interval 1 day``` if ```start_date``` is less than or equal to ```stop_date```, otherwise ```- interval 1 day```.
- ```sequence(start_timestamp, stop_timestamp, step_interval) → array<timestamp>```
Generate a sequence of timestamps from ```start_timestamps``` to ```stop_timestamps```, incrementing by ```step_interval```. The type of ```step_interval``` is ```CalendarInterval```.

## How was this patch tested?

Added unit tests.

Author: Vayda, Oleksandr: IT (PRG) <Oleksandr.Vayda@barclayscapital.com>

Closes #21155 from wajda/feature/array-api-sequence.
2018-06-27 11:52:31 +09:00
Maxim Gekk d08f53dc61 [SPARK-24605][SQL] size(null) returns null instead of -1
## What changes were proposed in this pull request?

In PR, I propose new behavior of `size(null)` under the config flag `spark.sql.legacy.sizeOfNull`. If the former one is disabled, the `size()` function returns `null` for `null` input. By default the `spark.sql.legacy.sizeOfNull` is enabled to keep backward compatibility with previous versions. In that case, `size(null)` returns `-1`.

## How was this patch tested?

Modified existing tests for the `size()` function to check new behavior (`null`) and old one (`-1`).

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21598 from MaxGekk/legacy-size-of-null.
2018-06-27 10:36:51 +08:00
Kris Mok 1b9368f7d4 [SPARK-24659][SQL] GenericArrayData.equals should respect element type differences
## What changes were proposed in this pull request?

Fix `GenericArrayData.equals`, so that it respects the actual types of the elements.
e.g. an instance that represents an `array<int>` and another instance that represents an `array<long>` should be considered incompatible, and thus should return false for `equals`.

`GenericArrayData` doesn't keep any schema information by itself, and rather relies on the Java objects referenced by its `array` field's elements to keep track of their own object types. So, the most straightforward way to respect their types is to call `equals` on the elements, instead of using Scala's `==` operator, which can have semantics that are not always desirable:
```
new java.lang.Integer(123) == new java.lang.Long(123L) // true in Scala
new java.lang.Integer(123).equals(new java.lang.Long(123L)) // false in Scala
```

## How was this patch tested?

Added unit test in `ComplexDataSuite`

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

Closes #21643 from rednaxelafx/fix-genericarraydata-equals.
2018-06-27 10:27:40 +08:00
Yuming Wang dcaa49ff1e [SPARK-24658][SQL] Remove workaround for ANTLR bug
## What changes were proposed in this pull request?

Issue antlr/antlr4#781 has already been fixed, so the workaround of extracting the pattern into a separate rule is no longer needed. The presto already removed it: https://github.com/prestodb/presto/pull/10744.

## How was this patch tested?

Existing tests

Author: Yuming Wang <yumwang@ebay.com>

Closes #21641 from wangyum/ANTLR-780.
2018-06-26 14:33:04 -07:00
Marek Novotny e07aee2165 [SPARK-24636][SQL] Type coercion of arrays for array_join function
## What changes were proposed in this pull request?
Presto's implementation accepts arbitrary arrays of primitive types as an input:

```
presto> SELECT array_join(ARRAY [1, 2, 3], ', ');
_col0
---------
1, 2, 3
(1 row)
```

This PR proposes to implement a type coercion rule for ```array_join``` function that converts arrays of primitive as well as non-primitive types to arrays of string.

## How was this patch tested?

New test cases add into:
- sql-tests/inputs/typeCoercion/native/arrayJoin.sql
- DataFrameFunctionsSuite.scala

Author: Marek Novotny <mn.mikke@gmail.com>

Closes #21620 from mn-mikke/SPARK-24636.
2018-06-26 09:51:55 +08:00
Bryan Cutler d48803bf64 [SPARK-24324][PYTHON][FOLLOWUP] Grouped Map positional conf should have deprecation note
## What changes were proposed in this pull request?

Followup to the discussion of the added conf in SPARK-24324 which allows assignment by column position only.  This conf is to preserve old behavior and will be removed in future releases, so it should have a note to indicate that.

## How was this patch tested?

NA

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #21637 from BryanCutler/arrow-groupedMap-conf-deprecate-followup-SPARK-24324.
2018-06-25 17:08:23 -07:00
Marco Gaido 594ac4f7b8 [SPARK-24633][SQL] Fix codegen when split is required for arrays_zip
## What changes were proposed in this pull request?

In function array_zip, when split is required by the high number of arguments, a codegen error can happen.

The PR fixes codegen for cases when splitting the code is required.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21621 from mgaido91/SPARK-24633.
2018-06-25 23:44:20 +08:00
Bryan Cutler a5849ad9a3 [SPARK-24324][PYTHON] Pandas Grouped Map UDF should assign result columns by name
## What changes were proposed in this pull request?

Currently, a `pandas_udf` of type `PandasUDFType.GROUPED_MAP` will assign the resulting columns based on index of the return pandas.DataFrame.  If a new DataFrame is returned and constructed using a dict, then the order of the columns could be arbitrary and be different than the defined schema for the UDF.  If the schema types still match, then no error will be raised and the user will see column names and column data mixed up.

This change will first try to assign columns using the return type field names.  If a KeyError occurs, then the column index is checked if it is string based. If so, then the error is raised as it is most likely a naming mistake, else it will fallback to assign columns by position and raise a TypeError if the field types do not match.

## How was this patch tested?

Added a test that returns a new DataFrame with column order different than the schema.

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #21427 from BryanCutler/arrow-grouped-map-mixesup-cols-SPARK-24324.
2018-06-24 09:28:46 +08:00
Maxim Gekk c7e2742f9b [SPARK-24190][SQL] Allow saving of JSON files in UTF-16 and UTF-32
## What changes were proposed in this pull request?

Currently, restrictions in JSONOptions for `encoding` and `lineSep` are the same for read and for write. For example, a requirement for `lineSep` in the code:

```
df.write.option("encoding", "UTF-32BE").json(file)
```
doesn't allow to skip `lineSep` and use its default value `\n` because it throws the exception:
```
equirement failed: The lineSep option must be specified for the UTF-32BE encoding
java.lang.IllegalArgumentException: requirement failed: The lineSep option must be specified for the UTF-32BE encoding
```

In the PR, I propose to separate JSONOptions in read and write, and make JSONOptions in write less restrictive.

## How was this patch tested?

Added new test for blacklisted encodings in read. And the `lineSep` option was removed in write for some tests.

Author: Maxim Gekk <maxim.gekk@databricks.com>
Author: Maxim Gekk <max.gekk@gmail.com>

Closes #21247 from MaxGekk/json-options-in-write.
2018-06-23 17:40:20 -07:00
Marek Novotny 92c2f00bd2 [SPARK-23934][SQL] Adding map_from_entries function
## What changes were proposed in this pull request?
The PR adds the `map_from_entries` function that returns a map created from the given array of entries.

## How was this patch tested?
New tests added into:
- `CollectionExpressionSuite`
- `DataFrameFunctionSuite`

## CodeGen Examples
### Primitive-type Keys and Values
```
val idf = Seq(
  Seq((1, 10), (2, 20), (3, 10)),
  Seq((1, 10), null, (2, 20))
).toDF("a")
idf.filter('a.isNotNull).select(map_from_entries('a)).debugCodegen
```
Result:
```
/* 042 */         boolean project_isNull_0 = false;
/* 043 */         MapData project_value_0 = null;
/* 044 */
/* 045 */         for (int project_idx_2 = 0; !project_isNull_0 && project_idx_2 < inputadapter_value_0.numElements(); project_idx_2++) {
/* 046 */           project_isNull_0 |= inputadapter_value_0.isNullAt(project_idx_2);
/* 047 */         }
/* 048 */         if (!project_isNull_0) {
/* 049 */           final int project_numEntries_0 = inputadapter_value_0.numElements();
/* 050 */
/* 051 */           final long project_keySectionSize_0 = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(project_numEntries_0, 4);
/* 052 */           final long project_valueSectionSize_0 = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(project_numEntries_0, 4);
/* 053 */           final long project_byteArraySize_0 = 8 + project_keySectionSize_0 + project_valueSectionSize_0;
/* 054 */           if (project_byteArraySize_0 > 2147483632) {
/* 055 */             final Object[] project_keys_0 = new Object[project_numEntries_0];
/* 056 */             final Object[] project_values_0 = new Object[project_numEntries_0];
/* 057 */
/* 058 */             for (int project_idx_1 = 0; project_idx_1 < project_numEntries_0; project_idx_1++) {
/* 059 */               InternalRow project_entry_1 = inputadapter_value_0.getStruct(project_idx_1, 2);
/* 060 */
/* 061 */               project_keys_0[project_idx_1] = project_entry_1.getInt(0);
/* 062 */               project_values_0[project_idx_1] = project_entry_1.getInt(1);
/* 063 */             }
/* 064 */
/* 065 */             project_value_0 = org.apache.spark.sql.catalyst.util.ArrayBasedMapData.apply(project_keys_0, project_values_0);
/* 066 */
/* 067 */           } else {
/* 068 */             final byte[] project_byteArray_0 = new byte[(int)project_byteArraySize_0];
/* 069 */             UnsafeMapData project_unsafeMapData_0 = new UnsafeMapData();
/* 070 */             Platform.putLong(project_byteArray_0, 16, project_keySectionSize_0);
/* 071 */             Platform.putLong(project_byteArray_0, 24, project_numEntries_0);
/* 072 */             Platform.putLong(project_byteArray_0, 24 + project_keySectionSize_0, project_numEntries_0);
/* 073 */             project_unsafeMapData_0.pointTo(project_byteArray_0, 16, (int)project_byteArraySize_0);
/* 074 */             ArrayData project_keyArrayData_0 = project_unsafeMapData_0.keyArray();
/* 075 */             ArrayData project_valueArrayData_0 = project_unsafeMapData_0.valueArray();
/* 076 */
/* 077 */             for (int project_idx_0 = 0; project_idx_0 < project_numEntries_0; project_idx_0++) {
/* 078 */               InternalRow project_entry_0 = inputadapter_value_0.getStruct(project_idx_0, 2);
/* 079 */
/* 080 */               project_keyArrayData_0.setInt(project_idx_0, project_entry_0.getInt(0));
/* 081 */               project_valueArrayData_0.setInt(project_idx_0, project_entry_0.getInt(1));
/* 082 */             }
/* 083 */
/* 084 */             project_value_0 = project_unsafeMapData_0;
/* 085 */           }
/* 086 */
/* 087 */         }
```
### Non-primitive-type Keys and Values
```
val sdf = Seq(
  Seq(("a", null), ("b", "bb"), ("c", "aa")),
  Seq(("a", "aa"), null, (null, "bb"))
).toDF("a")
sdf.filter('a.isNotNull).select(map_from_entries('a)).debugCodegen
```
Result:
```
/* 042 */         boolean project_isNull_0 = false;
/* 043 */         MapData project_value_0 = null;
/* 044 */
/* 045 */         for (int project_idx_1 = 0; !project_isNull_0 && project_idx_1 < inputadapter_value_0.numElements(); project_idx_1++) {
/* 046 */           project_isNull_0 |= inputadapter_value_0.isNullAt(project_idx_1);
/* 047 */         }
/* 048 */         if (!project_isNull_0) {
/* 049 */           final int project_numEntries_0 = inputadapter_value_0.numElements();
/* 050 */
/* 051 */           final Object[] project_keys_0 = new Object[project_numEntries_0];
/* 052 */           final Object[] project_values_0 = new Object[project_numEntries_0];
/* 053 */
/* 054 */           for (int project_idx_0 = 0; project_idx_0 < project_numEntries_0; project_idx_0++) {
/* 055 */             InternalRow project_entry_0 = inputadapter_value_0.getStruct(project_idx_0, 2);
/* 056 */
/* 057 */             if (project_entry_0.isNullAt(0)) {
/* 058 */               throw new RuntimeException("The first field from a struct (key) can't be null.");
/* 059 */             }
/* 060 */
/* 061 */             project_keys_0[project_idx_0] = project_entry_0.getUTF8String(0);
/* 062 */             project_values_0[project_idx_0] = project_entry_0.getUTF8String(1);
/* 063 */           }
/* 064 */
/* 065 */           project_value_0 = org.apache.spark.sql.catalyst.util.ArrayBasedMapData.apply(project_keys_0, project_values_0);
/* 066 */
/* 067 */         }
```

Author: Marek Novotny <mn.mikke@gmail.com>

Closes #21282 from mn-mikke/feature/array-api-map_from_entries-to-master.
2018-06-22 16:18:22 +09:00
Wenchen Fan dc8a6befa5 [SPARK-24588][SS] streaming join should require HashClusteredPartitioning from children
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/19080 we simplified the distribution/partitioning framework, and make all the join-like operators require `HashClusteredDistribution` from children. Unfortunately streaming join operator was missed.

This can cause wrong result. Think about
```
val input1 = MemoryStream[Int]
val input2 = MemoryStream[Int]

val df1 = input1.toDF.select('value as 'a, 'value * 2 as 'b)
val df2 = input2.toDF.select('value as 'a, 'value * 2 as 'b).repartition('b)
val joined = df1.join(df2, Seq("a", "b")).select('a)
```

The physical plan is
```
*(3) Project [a#5]
+- StreamingSymmetricHashJoin [a#5, b#6], [a#10, b#11], Inner, condition = [ leftOnly = null, rightOnly = null, both = null, full = null ], state info [ checkpoint = <unknown>, runId = 54e31fce-f055-4686-b75d-fcd2b076f8d8, opId = 0, ver = 0, numPartitions = 5], 0, state cleanup [ left = null, right = null ]
   :- Exchange hashpartitioning(a#5, b#6, 5)
   :  +- *(1) Project [value#1 AS a#5, (value#1 * 2) AS b#6]
   :     +- StreamingRelation MemoryStream[value#1], [value#1]
   +- Exchange hashpartitioning(b#11, 5)
      +- *(2) Project [value#3 AS a#10, (value#3 * 2) AS b#11]
         +- StreamingRelation MemoryStream[value#3], [value#3]
```

The left table is hash partitioned by `a, b`, while the right table is hash partitioned by `b`. This means, we may have a matching record that is in different partitions, which should be in the output but not.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21587 from cloud-fan/join.
2018-06-21 15:38:46 -07:00
Maxim Gekk 54fcaafb09 [SPARK-24571][SQL] Support Char literals
## What changes were proposed in this pull request?

In the PR, I propose to automatically convert a `Literal` with `Char` type to a `Literal` of `String` type. Currently, the following code:
```scala
val df = Seq("Amsterdam", "San Francisco", "London").toDF("city")
df.where($"city".contains('o')).show(false)
```
fails with the exception:
```
Unsupported literal type class java.lang.Character o
java.lang.RuntimeException: Unsupported literal type class java.lang.Character o
at org.apache.spark.sql.catalyst.expressions.Literal$.apply(literals.scala:78)
```
The PR fixes this issue by converting `char` to `string` of length `1`. I believe it makes sense to does not differentiate `char` and `string(1)` in _a unified, multi-language data platform_ like Spark which supports languages like Python/R.

Author: Maxim Gekk <maxim.gekk@databricks.com>
Author: Maxim Gekk <max.gekk@gmail.com>

Closes #21578 from MaxGekk/support-char-literals.
2018-06-20 23:38:37 -07:00
Huaxin Gao 9de11d3f90 [SPARK-23912][SQL] add array_distinct
## What changes were proposed in this pull request?

Add array_distinct to remove duplicate value from the array.

## How was this patch tested?

Add unit tests

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

Closes #21050 from huaxingao/spark-23912.
2018-06-21 12:24:53 +09:00
aokolnychyi c5a0d1132a [SPARK-24575][SQL] Prohibit window expressions inside WHERE and HAVING clauses
## What changes were proposed in this pull request?

As discussed [before](https://github.com/apache/spark/pull/19193#issuecomment-393726964), this PR prohibits window expressions inside WHERE and HAVING clauses.

## How was this patch tested?

This PR comes with a dedicated unit test.

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

Closes #21580 from aokolnychyi/spark-24575.
2018-06-20 18:57:13 +02:00
Xiao Li 9a75c18290 [SPARK-24542][SQL] UDF series UDFXPathXXXX allow users to pass carefully crafted XML to access arbitrary files
## What changes were proposed in this pull request?

UDF series UDFXPathXXXX allow users to pass carefully crafted XML to access arbitrary files. Spark does not have built-in access control. When users use the external access control library, users might bypass them and access the file contents.

This PR basically patches the Hive fix to Apache Spark. https://issues.apache.org/jira/browse/HIVE-18879

## How was this patch tested?

A unit test case

Author: Xiao Li <gatorsmile@gmail.com>

Closes #21549 from gatorsmile/xpathSecurity.
2018-06-18 20:17:04 -07:00
Liang-Chi Hsieh 8f225e055c [SPARK-24548][SQL] Fix incorrect schema of Dataset with tuple encoders
## What changes were proposed in this pull request?

When creating tuple expression encoders, we should give the serializer expressions of tuple items correct names, so we can have correct output schema when we use such tuple encoders.

## How was this patch tested?

Added test.

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

Closes #21576 from viirya/SPARK-24548.
2018-06-18 11:01:17 -07:00
Takeshi Yamamuro e219e692ef [SPARK-23772][SQL] Provide an option to ignore column of all null values or empty array during JSON schema inference
## What changes were proposed in this pull request?
This pr added a new JSON option `dropFieldIfAllNull ` to ignore column of all null values or empty array/struct during JSON schema inference.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>
Author: Xiangrui Meng <meng@databricks.com>

Closes #20929 from maropu/SPARK-23772.
2018-06-19 00:24:54 +08:00
James Yu c7c0b086a0 add one supported type missing from the javadoc
## What changes were proposed in this pull request?

The supported java.math.BigInteger type is not mentioned in the javadoc of Encoders.bean()

## How was this patch tested?

only Javadoc fix

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

Author: James Yu <james@ispot.tv>

Closes #21544 from yuj/master.
2018-06-15 21:04:04 -07:00
Kazuaki Ishizaki 90da7dc241 [SPARK-24452][SQL][CORE] Avoid possible overflow in int add or multiple
## What changes were proposed in this pull request?

This PR fixes possible overflow in int add or multiply. In particular, their overflows in multiply are detected by [Spotbugs](https://spotbugs.github.io/)

The following assignments may cause overflow in right hand side. As a result, the result may be negative.
```
long = int * int
long = int + int
```

To avoid this problem, this PR performs cast from int to long in right hand side.

## How was this patch tested?

Existing UTs.

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

Closes #21481 from kiszk/SPARK-24452.
2018-06-15 13:47:48 -07:00
Maxim Gekk b8f27ae3b3 [SPARK-24543][SQL] Support any type as DDL string for from_json's schema
## What changes were proposed in this pull request?

In the PR, I propose to support any DataType represented as DDL string for the from_json function. After the changes, it will be possible to specify `MapType` in SQL like:
```sql
select from_json('{"a":1, "b":2}', 'map<string, int>')
```
and in Scala (similar in other languages)
```scala
val in = Seq("""{"a": {"b": 1}}""").toDS()
val schema = "map<string, map<string, int>>"
val out = in.select(from_json($"value", schema, Map.empty[String, String]))
```

## How was this patch tested?

Added a couple sql tests and modified existing tests for Python and Scala. The former tests were modified because it is not imported for them in which format schema for `from_json` is provided.

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21550 from MaxGekk/from_json-ddl-schema.
2018-06-14 13:27:27 -07:00
Herman van Hovell 299d297e25 [SPARK-24500][SQL] Make sure streams are materialized during Tree transforms.
## What changes were proposed in this pull request?
If you construct catalyst trees using `scala.collection.immutable.Stream` you can run into situations where valid transformations do not seem to have any effect. There are two causes for this behavior:
- `Stream` is evaluated lazily. Note that default implementation will generally only evaluate a function for the first element (this makes testing a bit tricky).
- `TreeNode` and `QueryPlan` use side effects to detect if a tree has changed. Mapping over a stream is lazy and does not need to trigger this side effect. If this happens the node will invalidly assume that it did not change and return itself instead if the newly created node (this is for GC reasons).

This PR fixes this issue by forcing materialization on streams in `TreeNode` and `QueryPlan`.

## How was this patch tested?
Unit tests were added to `TreeNodeSuite` and `LogicalPlanSuite`. An integration test was added to the `PlannerSuite`

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

Closes #21539 from hvanhovell/SPARK-24500.
2018-06-13 07:09:48 -07:00
Arun Mahadevan 7703b46d28 [SPARK-24479][SS] Added config for registering streamingQueryListeners
## What changes were proposed in this pull request?

Currently a "StreamingQueryListener" can only be registered programatically. We could have a new config "spark.sql.streamingQueryListeners" similar to  "spark.sql.queryExecutionListeners" and "spark.extraListeners" for users to register custom streaming listeners.

## How was this patch tested?

New unit test and running example programs.

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

Author: Arun Mahadevan <arunm@apache.org>

Closes #21504 from arunmahadevan/SPARK-24480.
2018-06-13 20:43:16 +08:00
Li Jin 9786ce66c5 [SPARK-22239][SQL][PYTHON] Enable grouped aggregate pandas UDFs as window functions with unbounded window frames
## What changes were proposed in this pull request?
This PR enables using a grouped aggregate pandas UDFs as window functions. The semantics is the same as using SQL aggregation function as window functions.

```
       >>> from pyspark.sql.functions import pandas_udf, PandasUDFType
       >>> from pyspark.sql import Window
       >>> df = spark.createDataFrame(
       ...     [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)],
       ...     ("id", "v"))
       >>> pandas_udf("double", PandasUDFType.GROUPED_AGG)
       ... def mean_udf(v):
       ...     return v.mean()
       >>> w = Window.partitionBy('id')
       >>> df.withColumn('mean_v', mean_udf(df['v']).over(w)).show()
       +---+----+------+
       | id|   v|mean_v|
       +---+----+------+
       |  1| 1.0|   1.5|
       |  1| 2.0|   1.5|
       |  2| 3.0|   6.0|
       |  2| 5.0|   6.0|
       |  2|10.0|   6.0|
       +---+----+------+
```

The scope of this PR is somewhat limited in terms of:
(1) Only supports unbounded window, which acts essentially as group by.
(2) Only supports aggregation functions, not "transform" like window functions (n -> n mapping)

Both of these are left as future work. Especially, (1) needs careful thinking w.r.t. how to pass rolling window data to python efficiently. (2) is a bit easier but does require more changes therefore I think it's better to leave it as a separate PR.

## How was this patch tested?

WindowPandasUDFTests

Author: Li Jin <ice.xelloss@gmail.com>

Closes #21082 from icexelloss/SPARK-22239-window-udf.
2018-06-13 09:10:52 +08:00
Kazuaki Ishizaki ada28f2595 [SPARK-23933][SQL] Add map_from_arrays function
## What changes were proposed in this pull request?

The PR adds the SQL function `map_from_arrays`. The behavior of the function is based on Presto's `map`. Since SparkSQL already had a `map` function, we prepared the different name for this behavior.

This function returns returns a map from a pair of arrays for keys and values.

## How was this patch tested?

Added UTs

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

Closes #21258 from kiszk/SPARK-23933.
2018-06-12 12:31:22 -07:00
DylanGuedes f0ef1b311d [SPARK-23931][SQL] Adds arrays_zip function to sparksql
Signed-off-by: DylanGuedes <djmgguedesgmail.com>

## What changes were proposed in this pull request?

Addition of arrays_zip function to spark sql functions.

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
Unit tests that checks if the results are correct.

Author: DylanGuedes <djmgguedes@gmail.com>

Closes #21045 from DylanGuedes/SPARK-23931.
2018-06-12 11:57:25 -07:00
Marco Gaido f07c5064a3 [SPARK-24468][SQL] Handle negative scale when adjusting precision for decimal operations
## What changes were proposed in this pull request?

In SPARK-22036 we introduced the possibility to allow precision loss in arithmetic operations (according to the SQL standard). The implementation was drawn from Hive's one, where Decimals with a negative scale are not allowed in the operations.

The PR handles the case when the scale is negative, removing the assertion that it is not.

## How was this patch tested?

added UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21499 from mgaido91/SPARK-24468.
2018-06-08 18:51:56 -07:00
Bruce Robbins 1462bba4fd [SPARK-24119][SQL] Add interpreted execution to SortPrefix expression
## What changes were proposed in this pull request?

Implemented eval in SortPrefix expression.

## How was this patch tested?

- ran existing sbt SQL tests
- added unit test
- ran existing Python SQL tests
- manual tests: disabling codegen -- patching code to disable beyond what spark.sql.codegen.wholeStage=false can do -- and running sbt SQL tests

Author: Bruce Robbins <bersprockets@gmail.com>

Closes #21231 from bersprockets/sortprefixeval.
2018-06-08 13:27:52 +02:00
aokolnychyi 7297ae04d8 [SPARK-21896][SQL] Fix StackOverflow caused by window functions inside aggregate functions
## What changes were proposed in this pull request?

This PR explicitly prohibits window functions inside aggregates. Currently, this will cause StackOverflow during analysis. See PR #19193 for previous discussion.

## How was this patch tested?

This PR comes with a dedicated unit test.

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

Closes #21473 from aokolnychyi/fix-stackoverflow-window-funcs.
2018-06-04 13:28:16 -07:00
Yuming Wang 0be5aa2746 [SPARK-23903][SQL] Add support for date extract
## What changes were proposed in this pull request?

Add support for date `extract` function:
```sql
spark-sql> SELECT EXTRACT(YEAR FROM TIMESTAMP '2000-12-16 12:21:13');
2000
```
Supported field same as [Hive](https://github.com/apache/hive/blob/rel/release-2.3.3/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g#L308-L316): `YEAR`, `QUARTER`, `MONTH`, `WEEK`, `DAY`, `DAYOFWEEK`, `HOUR`, `MINUTE`, `SECOND`.

## How was this patch tested?

unit tests

Author: Yuming Wang <yumwang@ebay.com>

Closes #21479 from wangyum/SPARK-23903.
2018-06-04 10:16:13 -07:00
Xiao Li d2c3de7efc Revert "[SPARK-24369][SQL] Correct handling for multiple distinct aggregations having the same argument set"
This reverts commit 1e46f92f95.
2018-06-01 11:51:10 -07:00
Huaxin Gao 98909c398d [SPARK-23920][SQL] add array_remove to remove all elements that equal element from array
## What changes were proposed in this pull request?

add array_remove to remove all elements that equal element from array

## How was this patch tested?

add unit tests

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

Closes #21069 from huaxingao/spark-23920.
2018-05-31 22:04:26 -07:00
Yuming Wang cc976f6cb8 [SPARK-23900][SQL] format_number support user specifed format as argument
## What changes were proposed in this pull request?

`format_number` support user specifed format as argument. For example:
```sql
spark-sql> SELECT format_number(12332.123456, '##################.###');
12332.123
```

## How was this patch tested?

unit test

Author: Yuming Wang <yumwang@ebay.com>

Closes #21010 from wangyum/SPARK-23900.
2018-05-31 11:38:23 -07:00
Marco Gaido 24ef7fbfa9 [SPARK-24276][SQL] Order of literals in IN should not affect semantic equality
## What changes were proposed in this pull request?

When two `In` operators are created with the same list of values, but different order, we are considering them as semantically different. This is wrong, since they have the same semantic meaning.

The PR adds a canonicalization rule which orders the literals in the `In` operator so the semantic equality works properly.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21331 from mgaido91/SPARK-24276.
2018-05-30 15:31:40 -07:00
Marco Gaido 1b36f14889 [SPARK-23901][SQL] Add masking functions
## What changes were proposed in this pull request?

The PR adds the masking function as they are described in Hive's documentation: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF#LanguageManualUDF-DataMaskingFunctions.
This means that only `string`s are accepted as parameter for the masking functions.

## How was this patch tested?

added UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21246 from mgaido91/SPARK-23901.
2018-05-30 11:18:04 -07:00
Takeshi Yamamuro 1e46f92f95 [SPARK-24369][SQL] Correct handling for multiple distinct aggregations having the same argument set
## What changes were proposed in this pull request?
This pr fixed an issue when having multiple distinct aggregations having the same argument set, e.g.,
```
scala>: paste
val df = sql(
  s"""SELECT corr(DISTINCT x, y), corr(DISTINCT y, x), count(*)
     | FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y)
   """.stripMargin)

java.lang.RuntimeException
You hit a query analyzer bug. Please report your query to Spark user mailing list.
```
The root cause is that `RewriteDistinctAggregates` can't detect multiple distinct aggregations if they have the same argument set. This pr modified code so that `RewriteDistinctAggregates` could count the number of aggregate expressions with `isDistinct=true`.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21443 from maropu/SPARK-24369.
2018-05-31 00:23:25 +08:00
DB Tsai 900bc1f7dc
[SPARK-24371][SQL] Added isInCollection in DataFrame API for Scala and Java.
## What changes were proposed in this pull request?

Implemented **`isInCollection `** in DataFrame API for both Scala and Java, so users can do

```scala
val profileDF = Seq(
  Some(1), Some(2), Some(3), Some(4),
  Some(5), Some(6), Some(7), None
).toDF("profileID")

val validUsers: Seq[Any] = Seq(6, 7.toShort, 8L, "3")

val result = profileDF.withColumn("isValid", $"profileID". isInCollection(validUsers))

result.show(10)
"""
+---------+-------+
|profileID|isValid|
+---------+-------+
|        1|  false|
|        2|  false|
|        3|   true|
|        4|  false|
|        5|  false|
|        6|   true|
|        7|   true|
|     null|   null|
+---------+-------+
 """.stripMargin
```
## How was this patch tested?

Several unit tests are added.

Author: DB Tsai <d_tsai@apple.com>

Closes #21416 from dbtsai/optimize-set.
2018-05-29 10:22:18 -07:00
Miles Yucht d440699192 [SPARK-24381][TESTING] Add unit tests for NOT IN subquery around null values
## What changes were proposed in this pull request?
This PR adds several unit tests along the `cols NOT IN (subquery)` pathway. There are a scattering of tests here and there which cover this codepath, but there doesn't seem to be a unified unit test of the correctness of null-aware anti joins anywhere. I have also added a brief explanation of how this expression behaves in SubquerySuite. Lastly, I made some clarifying changes in the NOT IN pathway in RewritePredicateSubquery.

## How was this patch tested?
Added unit tests! There should be no behavioral change in this PR.

Author: Miles Yucht <miles@databricks.com>

Closes #21425 from mgyucht/spark-24381.
2018-05-26 20:42:23 -07:00
Maxim Gekk 1b1528a504 [SPARK-24366][SQL] Improving of error messages for type converting
## What changes were proposed in this pull request?

Currently, users are getting the following error messages on type conversions:

```
scala.MatchError: test (of class java.lang.String)
```

The message doesn't give any clues to the users where in the schema the error happened. In this PR, I would like to improve the error message like:

```
The value (test) of the type (java.lang.String) cannot be converted to struct<f1:int>
```

## How was this patch tested?

Added tests for converting of wrong values to `struct`, `map`, `array`, `string` and `decimal`.

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21410 from MaxGekk/type-conv-error.
2018-05-25 15:42:46 -07:00
Maxim Gekk 64fad0b519 [SPARK-24244][SPARK-24368][SQL] Passing only required columns to the CSV parser
## What changes were proposed in this pull request?

uniVocity parser allows to specify only required column names or indexes for [parsing](https://www.univocity.com/pages/parsers-tutorial) like:

```
// Here we select only the columns by their indexes.
// The parser just skips the values in other columns
parserSettings.selectIndexes(4, 0, 1);
CsvParser parser = new CsvParser(parserSettings);
```
In this PR, I propose to extract indexes from required schema and pass them into the CSV parser. Benchmarks show the following improvements in parsing of 1000 columns:

```
Select 100 columns out of 1000: x1.76
Select 1 column out of 1000: x2
```

**Note**: Comparing to current implementation, the changes can return different result for malformed rows in the `DROPMALFORMED` and `FAILFAST` modes if only subset of all columns is requested. To have previous behavior, set `spark.sql.csv.parser.columnPruning.enabled` to `false`.

## How was this patch tested?

It was tested by new test which selects 3 columns out of 15, by existing tests and by new benchmarks.

Author: Maxim Gekk <maxim.gekk@databricks.com>
Author: Maxim Gekk <max.gekk@gmail.com>

Closes #21415 from MaxGekk/csv-column-pruning2.
2018-05-24 21:38:04 -07:00
Gengliang Wang 3b20b34ab7 [SPARK-24367][SQL] Parquet: use JOB_SUMMARY_LEVEL instead of deprecated flag ENABLE_JOB_SUMMARY
## What changes were proposed in this pull request?

In current parquet version,the conf ENABLE_JOB_SUMMARY is deprecated.

When writing to Parquet files, the warning message
```WARN org.apache.parquet.hadoop.ParquetOutputFormat: Setting parquet.enable.summary-metadata is deprecated, please use parquet.summary.metadata.level```
keeps showing up.

From https://github.com/apache/parquet-mr/blame/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java#L164 we can see that we should use JOB_SUMMARY_LEVEL.

## How was this patch tested?

Unit test

Author: Gengliang Wang <gengliang.wang@databricks.com>

Closes #21411 from gengliangwang/summaryLevel.
2018-05-25 11:16:35 +08:00
Yuming Wang 0d89943449 [SPARK-24378][SQL] Fix date_trunc function incorrect examples
## What changes were proposed in this pull request?

Fix `date_trunc` function incorrect examples.

## How was this patch tested?

N/A

Author: Yuming Wang <yumwang@ebay.com>

Closes #21423 from wangyum/SPARK-24378.
2018-05-24 23:38:50 +08:00
Vayda, Oleksandr: IT (PRG) 230f144197 [SPARK-24350][SQL] Fixes ClassCastException in the "array_position" function
## What changes were proposed in this pull request?

### Fixes `ClassCastException` in the `array_position` function - [SPARK-24350](https://issues.apache.org/jira/browse/SPARK-24350)
When calling `array_position` function with a wrong type of the 1st argument an `AnalysisException` should be thrown instead of `ClassCastException`

Example:

```sql
select array_position('foo', 'bar')
```

```
java.lang.ClassCastException: org.apache.spark.sql.types.StringType$ cannot be cast to org.apache.spark.sql.types.ArrayType
	at org.apache.spark.sql.catalyst.expressions.ArrayPosition.inputTypes(collectionOperations.scala:1398)
	at org.apache.spark.sql.catalyst.expressions.ExpectsInputTypes$class.checkInputDataTypes(ExpectsInputTypes.scala:44)
	at org.apache.spark.sql.catalyst.expressions.ArrayPosition.checkInputDataTypes(collectionOperations.scala:1401)
	at org.apache.spark.sql.catalyst.expressions.Expression.resolved$lzycompute(Expression.scala:168)
	at org.apache.spark.sql.catalyst.expressions.Expression.resolved(Expression.scala:168)
	at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAliases$$anonfun$org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveAliases$$assignAliases$1$$anonfun$apply$3.applyOrElse(Analyzer.scala:256)
	at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAliases$$anonfun$org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveAliases$$assignAliases$1$$anonfun$apply$3.applyOrElse(Analyzer.scala:252)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:289)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:289)
	at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
	at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:288)
```

## How was this patch tested?

unit test

Author: Vayda, Oleksandr: IT (PRG) <Oleksandr.Vayda@barclayscapital.com>

Closes #21401 from wajda/SPARK-24350-array_position-error-fix.
2018-05-23 17:22:52 -07:00
Xiao Li 5a5a868dc4 Revert "[SPARK-24244][SQL] Passing only required columns to the CSV parser"
This reverts commit 8086acc2f6.
2018-05-23 11:51:13 -07:00
Liang-Chi Hsieh a40ffc656d [SPARK-23711][SQL] Add fallback generator for UnsafeProjection
## What changes were proposed in this pull request?

Add fallback logic for `UnsafeProjection`. In production we can try to create unsafe projection using codegen implementation. Once any compile error happens, it fallbacks to interpreted implementation.

## How was this patch tested?

Added test.

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

Closes #21106 from viirya/SPARK-23711.
2018-05-23 22:40:52 +08:00
Vayda, Oleksandr: IT (PRG) bc6ea614ad [SPARK-24348][SQL] "element_at" error fix
## What changes were proposed in this pull request?

### Fixes a `scala.MatchError` in the `element_at` operation - [SPARK-24348](https://issues.apache.org/jira/browse/SPARK-24348)

When calling `element_at` with a wrong first operand type an `AnalysisException` should be thrown instead of `scala.MatchError`

*Example:*
```sql
select element_at('foo', 1)
```

results in:
```
scala.MatchError: StringType (of class org.apache.spark.sql.types.StringType$)
	at org.apache.spark.sql.catalyst.expressions.ElementAt.inputTypes(collectionOperations.scala:1469)
	at org.apache.spark.sql.catalyst.expressions.ExpectsInputTypes$class.checkInputDataTypes(ExpectsInputTypes.scala:44)
	at org.apache.spark.sql.catalyst.expressions.ElementAt.checkInputDataTypes(collectionOperations.scala:1478)
	at org.apache.spark.sql.catalyst.expressions.Expression.resolved$lzycompute(Expression.scala:168)
	at org.apache.spark.sql.catalyst.expressions.Expression.resolved(Expression.scala:168)
	at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAliases$$anonfun$org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveAliases$$assignAliases$1$$anonfun$apply$3.applyOrElse(Analyzer.scala:256)
	at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAliases$$anonfun$org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveAliases$$assignAliases$1$$anonfun$apply$3.applyOrElse(Analyzer.scala:252)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:289)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:289)
	at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
	at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:288)
```

## How was this patch tested?

unit tests

Author: Vayda, Oleksandr: IT (PRG) <Oleksandr.Vayda@barclayscapital.com>

Closes #21395 from wajda/SPARK-24348-element_at-error-fix.
2018-05-22 13:01:07 -07:00
Liang-Chi Hsieh f9f055afa4 [SPARK-24121][SQL] Add API for handling expression code generation
## What changes were proposed in this pull request?

This patch tries to implement this [proposal](https://github.com/apache/spark/pull/19813#issuecomment-354045400) to add an API for handling expression code generation. It should allow us to manipulate how to generate codes for expressions.

In details, this adds an new abstraction `CodeBlock` to `JavaCode`. `CodeBlock` holds the code snippet and inputs for generating actual java code.

For example, in following java code:

```java
  int ${variable} = 1;
  boolean ${isNull} = ${CodeGenerator.defaultValue(BooleanType)};
```

`variable`, `isNull` are two `VariableValue` and `CodeGenerator.defaultValue(BooleanType)` is a string. They are all inputs to this code block and held by `CodeBlock` representing this code.

For codegen, we provide a specified string interpolator `code`, so you can define a code like this:
```scala
  val codeBlock =
    code"""
         |int ${variable} = 1;
         |boolean ${isNull} = ${CodeGenerator.defaultValue(BooleanType)};
        """.stripMargin
  // Generates actual java code.
  codeBlock.toString
```

Because those inputs are held separately in `CodeBlock` before generating code, we can safely manipulate them, e.g., replacing statements to aliased variables, etc..

## How was this patch tested?

Added tests.

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

Closes #21193 from viirya/SPARK-24121.
2018-05-23 01:50:22 +08:00
Maxim Gekk 8086acc2f6 [SPARK-24244][SQL] Passing only required columns to the CSV parser
## What changes were proposed in this pull request?

uniVocity parser allows to specify only required column names or indexes for [parsing](https://www.univocity.com/pages/parsers-tutorial) like:

```
// Here we select only the columns by their indexes.
// The parser just skips the values in other columns
parserSettings.selectIndexes(4, 0, 1);
CsvParser parser = new CsvParser(parserSettings);
```
In this PR, I propose to extract indexes from required schema and pass them into the CSV parser. Benchmarks show the following improvements in parsing of 1000 columns:

```
Select 100 columns out of 1000: x1.76
Select 1 column out of 1000: x2
```

**Note**: Comparing to current implementation, the changes can return different result for malformed rows in the `DROPMALFORMED` and `FAILFAST` modes if only subset of all columns is requested. To have previous behavior, set `spark.sql.csv.parser.columnPruning.enabled` to `false`.

## How was this patch tested?

It was tested by new test which selects 3 columns out of 15, by existing tests and by new benchmarks.

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21296 from MaxGekk/csv-column-pruning.
2018-05-22 22:07:32 +08:00
Marco Gaido d3d1807315 [SPARK-24313][SQL] Fix collection operations' interpreted evaluation for complex types
## What changes were proposed in this pull request?

The interpreted evaluation of several collection operations works only for simple datatypes. For complex data types, for instance, `array_contains` it returns always `false`. The list of the affected functions is `array_contains`, `array_position`, `element_at` and `GetMapValue`.

The PR fixes the behavior for all the datatypes.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21361 from mgaido91/SPARK-24313.
2018-05-22 21:08:49 +08:00
Kris Mok 952e4d1c83 [SPARK-24321][SQL] Extract common code from Divide/Remainder to a base trait
## What changes were proposed in this pull request?

Extract common code from `Divide`/`Remainder` to a new base trait, `DivModLike`.

Further refactoring to make `Pmod` work with `DivModLike` is to be done as a separate task.

## How was this patch tested?

Existing tests in `ArithmeticExpressionSuite` covers the functionality.

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

Closes #21367 from rednaxelafx/catalyst-divmod.
2018-05-22 19:12:30 +08:00
Wenchen Fan 03e90f65bf [SPARK-24250][SQL] support accessing SQLConf inside tasks
re-submit https://github.com/apache/spark/pull/21299 which broke build.

A few new commits are added to fix the SQLConf problem in `JsonSchemaInference.infer`, and prevent us to access `SQLConf` in DAGScheduler event loop thread.

## What changes were proposed in this pull request?

Previously in #20136 we decided to forbid tasks to access `SQLConf`, because it doesn't work and always give you the default conf value. In #21190 we fixed the check and all the places that violate it.

Currently the pattern of accessing configs at the executor side is: read the configs at the driver side, then access the variables holding the config values in the RDD closure, so that they will be serialized to the executor side. Something like
```
val someConf = conf.getXXX
child.execute().mapPartitions {
  if (someConf == ...) ...
  ...
}
```

However, this pattern is hard to apply if the config needs to be propagated via a long call stack. An example is `DataType.sameType`, and see how many changes were made in #21190 .

When it comes to code generation, it's even worse. I tried it locally and we need to change a ton of files to propagate configs to code generators.

This PR proposes to allow tasks to access `SQLConf`. The idea is, we can save all the SQL configs to job properties when an SQL execution is triggered. At executor side we rebuild the `SQLConf` from job properties.

## How was this patch tested?

a new test suite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21376 from cloud-fan/config.
2018-05-22 00:19:18 +08:00
Marek Novotny a6e883feb3 [SPARK-23935][SQL] Adding map_entries function
## What changes were proposed in this pull request?

This PR adds `map_entries` function that returns an unordered array of all entries in the given map.

## How was this patch tested?

New tests added into:
- `CollectionExpressionSuite`
- `DataFrameFunctionsSuite`

## CodeGen examples
### Primitive types
```
val df = Seq(Map(1 -> 5, 2 -> 6)).toDF("m")
df.filter('m.isNotNull).select(map_entries('m)).debugCodegen
```
Result:
```
/* 042 */         boolean project_isNull_0 = false;
/* 043 */
/* 044 */         ArrayData project_value_0 = null;
/* 045 */
/* 046 */         final int project_numElements_0 = inputadapter_value_0.numElements();
/* 047 */         final ArrayData project_keys_0 = inputadapter_value_0.keyArray();
/* 048 */         final ArrayData project_values_0 = inputadapter_value_0.valueArray();
/* 049 */
/* 050 */         final long project_size_0 = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(
/* 051 */           project_numElements_0,
/* 052 */           32);
/* 053 */         if (project_size_0 > 2147483632) {
/* 054 */           final Object[] project_internalRowArray_0 = new Object[project_numElements_0];
/* 055 */           for (int z = 0; z < project_numElements_0; z++) {
/* 056 */             project_internalRowArray_0[z] = new org.apache.spark.sql.catalyst.expressions.GenericInternalRow(new Object[]{project_keys_0.getInt(z), project_values_0.getInt(z)});
/* 057 */           }
/* 058 */           project_value_0 = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_internalRowArray_0);
/* 059 */
/* 060 */         } else {
/* 061 */           final byte[] project_arrayBytes_0 = new byte[(int)project_size_0];
/* 062 */           UnsafeArrayData project_unsafeArrayData_0 = new UnsafeArrayData();
/* 063 */           Platform.putLong(project_arrayBytes_0, 16, project_numElements_0);
/* 064 */           project_unsafeArrayData_0.pointTo(project_arrayBytes_0, 16, (int)project_size_0);
/* 065 */
/* 066 */           final int project_structsOffset_0 = UnsafeArrayData.calculateHeaderPortionInBytes(project_numElements_0) + project_numElements_0 * 8;
/* 067 */           UnsafeRow project_unsafeRow_0 = new UnsafeRow(2);
/* 068 */           for (int z = 0; z < project_numElements_0; z++) {
/* 069 */             long offset = project_structsOffset_0 + z * 24L;
/* 070 */             project_unsafeArrayData_0.setLong(z, (offset << 32) + 24L);
/* 071 */             project_unsafeRow_0.pointTo(project_arrayBytes_0, 16 + offset, 24);
/* 072 */             project_unsafeRow_0.setInt(0, project_keys_0.getInt(z));
/* 073 */             project_unsafeRow_0.setInt(1, project_values_0.getInt(z));
/* 074 */           }
/* 075 */           project_value_0 = project_unsafeArrayData_0;
/* 076 */
/* 077 */         }
```
### Non-primitive types
```
val df = Seq(Map("a" -> "foo", "b" -> null)).toDF("m")
df.filter('m.isNotNull).select(map_entries('m)).debugCodegen
```
Result:
```
/* 042 */         boolean project_isNull_0 = false;
/* 043 */
/* 044 */         ArrayData project_value_0 = null;
/* 045 */
/* 046 */         final int project_numElements_0 = inputadapter_value_0.numElements();
/* 047 */         final ArrayData project_keys_0 = inputadapter_value_0.keyArray();
/* 048 */         final ArrayData project_values_0 = inputadapter_value_0.valueArray();
/* 049 */
/* 050 */         final Object[] project_internalRowArray_0 = new Object[project_numElements_0];
/* 051 */         for (int z = 0; z < project_numElements_0; z++) {
/* 052 */           project_internalRowArray_0[z] = new org.apache.spark.sql.catalyst.expressions.GenericInternalRow(new Object[]{project_keys_0.getUTF8String(z), project_values_0.getUTF8String(z)});
/* 053 */         }
/* 054 */         project_value_0 = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_internalRowArray_0);
```

Author: Marek Novotny <mn.mikke@gmail.com>

Closes #21236 from mn-mikke/feature/array-api-map_entries-to-master.
2018-05-21 23:14:03 +09:00
Wenchen Fan 000e25ae79 Revert "[SPARK-24250][SQL] support accessing SQLConf inside tasks"
This reverts commit dd37529a8d.
2018-05-20 16:13:42 +08:00
Wenchen Fan dd37529a8d [SPARK-24250][SQL] support accessing SQLConf inside tasks
## What changes were proposed in this pull request?

Previously in #20136 we decided to forbid tasks to access `SQLConf`, because it doesn't work and always give you the default conf value. In #21190 we fixed the check and all the places that violate it.

Currently the pattern of accessing configs at the executor side is: read the configs at the driver side, then access the variables holding the config values in the RDD closure, so that they will be serialized to the executor side. Something like
```
val someConf = conf.getXXX
child.execute().mapPartitions {
  if (someConf == ...) ...
  ...
}
```

However, this pattern is hard to apply if the config needs to be propagated via a long call stack. An example is `DataType.sameType`, and see how many changes were made in #21190 .

When it comes to code generation, it's even worse. I tried it locally and we need to change a ton of files to propagate configs to code generators.

This PR proposes to allow tasks to access `SQLConf`. The idea is, we can save all the SQL configs to job properties when an SQL execution is triggered. At executor side we rebuild the `SQLConf` from job properties.

## How was this patch tested?

a new test suite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21299 from cloud-fan/config.
2018-05-19 18:51:02 +08:00
Marcelo Vanzin ed7ba7db8f [SPARK-23850][SQL] Add separate config for SQL options redaction.
The old code was relying on a core configuration and extended its
default value to include things that redact desired things in the
app's environment. Instead, add a SQL-specific option for which
options to redact, and apply both the core and SQL-specific rules
when redacting the options in the save command.

This is a little sub-optimal since it adds another config, but it
retains the current default behavior.

While there I also fixed a typo and a couple of minor config API
usage issues in the related redaction option that SQL already had.

Tested with existing unit tests, plus checking the env page on
a shell UI.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #21158 from vanzin/SPARK-23850.
2018-05-18 11:14:22 -07:00
jinxing 8a837bf4f3 [SPARK-24193] create TakeOrderedAndProjectExec only when the limit number is below spark.sql.execution.topKSortFallbackThreshold.
## What changes were proposed in this pull request?

Physical plan of `select colA from t order by colB limit M` is `TakeOrderedAndProject`;
Currently `TakeOrderedAndProject` sorts data in memory, see https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala#L158
We can add a config – if the number of limit (M) is too big, we can sort by disk. Thus memory issue can be resolved.

## How was this patch tested?

Test added

Author: jinxing <jinxing6042@126.com>

Closes #21252 from jinxing64/SPARK-24193.
2018-05-17 22:29:18 +08:00
Marco Gaido 69350aa2f0 [SPARK-23922][SQL] Add arrays_overlap function
## What changes were proposed in this pull request?

The PR adds the function `arrays_overlap`. This function returns `true` if the input arrays contain a non-null common element; if not, it returns `null` if any of the arrays contains a `null` element, `false` otherwise.

## How was this patch tested?

added UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21028 from mgaido91/SPARK-23922.
2018-05-17 20:45:32 +08:00
Florent Pépin 3e66350c24 [SPARK-23925][SQL] Add array_repeat collection function
## What changes were proposed in this pull request?

The PR adds a new collection function, array_repeat. As there already was a function repeat with the same signature, with the only difference being the expected return type (String instead of Array), the new function is called array_repeat to distinguish.
The behaviour of the function is based on Presto's one.

The function creates an array containing a given element repeated the requested number of times.

## How was this patch tested?

New unit tests added into:
- CollectionExpressionsSuite
- DataFrameFunctionsSuite

Author: Florent Pépin <florentpepin.92@gmail.com>
Author: Florent Pépin <florent.pepin14@imperial.ac.uk>

Closes #21208 from pepinoflo/SPARK-23925.
2018-05-17 13:31:14 +09:00
Wenchen Fan 943493b165 Revert "[SPARK-22938][SQL][FOLLOWUP] Assert that SQLConf.get is acces…
…sed only on the driver"

This reverts commit a4206d58e0.

This is from https://github.com/apache/spark/pull/21299 and to ease the review of it.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21341 from cloud-fan/revert.
2018-05-16 22:01:24 +08:00
Jose Torres 3fabbc5762 [SPARK-24040][SS] Support single partition aggregates in continuous processing.
## What changes were proposed in this pull request?

Support aggregates with exactly 1 partition in continuous processing.

A few small tweaks are needed to make this work:

* Replace currentEpoch tracking with an ThreadLocal. This means that current epoch is scoped to a task rather than a node, but I think that's sustainable even once we add shuffle.
* Add a new testing-only flag to disable the UnsupportedOperationChecker whitelist of allowed continuous processing nodes. I think this is preferable to writing a pile of custom logic to enforce that there is in fact only 1 partition; we plan to support multi-partition aggregates before the next Spark release, so we'd just have to tear that logic back out.
* Restart continuous processing queries from the first available uncommitted epoch, rather than one that's guaranteed to be unused. This is required for stateful operators to overwrite partial state from the previous attempt at the epoch, and there was no specific motivation for the original strategy. In another PR before stabilizing the StreamWriter API, we'll need to narrow down and document more precise semantic guarantees for the epoch IDs.
* We need a single-partition ContinuousMemoryStream. The way MemoryStream is constructed means it can't be a text option like it is for rate source, unfortunately.

## How was this patch tested?

new unit tests

Author: Jose Torres <torres.joseph.f+github@gmail.com>

Closes #21239 from jose-torres/withAggr.
2018-05-15 10:25:29 -07:00
maryannxue 80c6d35a3e [SPARK-24035][SQL] SQL syntax for Pivot - fix antlr warning
## What changes were proposed in this pull request?

1. Change antlr rule to fix the warning.
2. Add PIVOT/LATERAL check in AstBuilder with a more meaningful error message.

## How was this patch tested?

1. Add a counter case in `PlanParserSuite.test("lateral view")`

Author: maryannxue <maryann.xue@gmail.com>

Closes #21324 from maryannxue/spark-24035-fix.
2018-05-14 23:34:42 -07:00
Maxim Gekk 8cd83acf40 [SPARK-24027][SQL] Support MapType with StringType for keys as the root type by from_json
## What changes were proposed in this pull request?

Currently, the from_json function support StructType or ArrayType as the root type. The PR allows to specify MapType(StringType, DataType) as the root type additionally to mentioned types. For example:

```scala
import org.apache.spark.sql.types._
val schema = MapType(StringType, IntegerType)
val in = Seq("""{"a": 1, "b": 2, "c": 3}""").toDS()
in.select(from_json($"value", schema, Map[String, String]())).collect()
```
```
res1: Array[org.apache.spark.sql.Row] = Array([Map(a -> 1, b -> 2, c -> 3)])
```

## How was this patch tested?

It was checked by new tests for the map type with integer type and struct type as value types. Also roundtrip tests like from_json(to_json) and to_json(from_json) for MapType are added.

Author: Maxim Gekk <maxim.gekk@databricks.com>
Author: Maxim Gekk <max.gekk@gmail.com>

Closes #21108 from MaxGekk/from_json-map-type.
2018-05-14 14:05:42 -07:00
Shixiong Zhu c26f673252 [SPARK-24246][SQL] Improve AnalysisException by setting the cause when it's available
## What changes were proposed in this pull request?

If there is an exception, it's better to set it as the cause of AnalysisException since the exception may contain useful debug information.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #21297 from zsxwing/SPARK-24246.
2018-05-14 11:37:57 -07:00
Cody Allen 32acfa78c6 Improve implicitNotFound message for Encoder
The `implicitNotFound` message for `Encoder` doesn't mention the name of
the type for which it can't find an encoder. Furthermore, it covers up
the fact that `Encoder` is the name of the relevant type class.
Hopefully this new message provides a little more specific type detail
while still giving the general message about which types are supported.

## What changes were proposed in this pull request?

Augment the existing message to mention that it's looking for an `Encoder` and what the type of the encoder is.

For example instead of:

```
Unable to find encoder for type stored in a Dataset.  Primitive types (Int, String, etc) and Product types (case classes) are supported by importing spark.implicits._  Support for serializing other types will be added in future releases.
```

return this message:

```
Unable to find encoder for type Exception. An implicit Encoder[Exception] is needed to store Exception instances in a Dataset. Primitive types (Int, String, etc) and Product types (ca
se classes) are supported by importing spark.implicits._  Support for serializing other types will be added in future releases.
```

## How was this patch tested?

It was tested manually in the Scala REPL, since triggering this in a test would cause a compilation error.

```
scala> implicitly[Encoder[Exception]]
<console>:51: error: Unable to find encoder for type Exception. An implicit Encoder[Exception] is needed to store Exception instances in a Dataset. Primitive types (Int, String, etc) and Product types (ca
se classes) are supported by importing spark.implicits._  Support for serializing other types will be added in future releases.
       implicitly[Encoder[Exception]]
                 ^
```

Author: Cody Allen <ceedubs@gmail.com>

Closes #20869 from ceedubs/encoder-implicit-msg.
2018-05-12 14:35:40 -05:00
aditkumar 92f6f52ff0 [MINOR][DOCS] Documenting months_between direction
## What changes were proposed in this pull request?

It's useful to know what relationship between date1 and date2 results in a positive number.

Author: aditkumar <aditkumar@gmail.com>
Author: Adit Kumar <aditkumar@gmail.com>

Closes #20787 from aditkumar/master.
2018-05-11 14:42:23 -05:00
Wenchen Fan a4206d58e0 [SPARK-22938][SQL][FOLLOWUP] Assert that SQLConf.get is accessed only on the driver
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/20136 . #20136 didn't really work because in the test, we are using local backend, which shares the driver side `SparkEnv`, so `SparkEnv.get.executorId == SparkContext.DRIVER_IDENTIFIER` doesn't work.

This PR changes the check to `TaskContext.get != null`, and move the check to `SQLConf.get`, and fix all the places that violate this check:
* `InMemoryTableScanExec#createAndDecompressColumn` is executed inside `rdd.map`, we can't access `conf.offHeapColumnVectorEnabled` there. https://github.com/apache/spark/pull/21223 merged
* `DataType#sameType` may be executed in executor side, for things like json schema inference, so we can't call `conf.caseSensitiveAnalysis` there. This contributes to most of the code changes, as we need to add `caseSensitive` parameter to a lot of methods.
* `ParquetFilters` is used in the file scan function, which is executed in executor side, so we can't can't call `conf.parquetFilterPushDownDate` there. https://github.com/apache/spark/pull/21224 merged
* `WindowExec#createBoundOrdering` is called on executor side, so we can't use `conf.sessionLocalTimezone` there. https://github.com/apache/spark/pull/21225 merged
* `JsonToStructs` can be serialized to executors and evaluate, we should not call `SQLConf.get.getConf(SQLConf.FROM_JSON_FORCE_NULLABLE_SCHEMA)` in the body. https://github.com/apache/spark/pull/21226 merged

## How was this patch tested?

existing test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21190 from cloud-fan/minor.
2018-05-11 09:01:40 +08:00
Maxim Gekk f4fed05121 [SPARK-24171] Adding a note for non-deterministic functions
## What changes were proposed in this pull request?

I propose to add a clear statement for functions like `collect_list()` about non-deterministic behavior of such functions. The behavior must be taken into account by user while creating and running queries.

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21228 from MaxGekk/deterministic-comments.
2018-05-10 09:44:49 -07:00
Marco Gaido 94d6714482 [SPARK-23907][SQL] Add regr_* functions
## What changes were proposed in this pull request?

The PR introduces regr_slope, regr_intercept, regr_r2, regr_sxx, regr_syy, regr_sxy, regr_avgx, regr_avgy, regr_count.

The implementation of this functions mirrors Hive's one in HIVE-15978.

## How was this patch tested?

added UT (values compared with Hive)

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21054 from mgaido91/SPARK-23907.
2018-05-10 20:38:52 +09:00
Ryan Blue cac9b1dea1 [SPARK-23972][BUILD][SQL] Update Parquet to 1.10.0.
## What changes were proposed in this pull request?

This updates Parquet to 1.10.0 and updates the vectorized path for buffer management changes. Parquet 1.10.0 uses ByteBufferInputStream instead of byte arrays in encoders. This allows Parquet to break allocations into smaller chunks that are better for garbage collection.

## How was this patch tested?

Existing Parquet tests. Running in production at Netflix for about 3 months.

Author: Ryan Blue <blue@apache.org>

Closes #21070 from rdblue/SPARK-23972-update-parquet-to-1.10.0.
2018-05-09 12:27:32 +08:00
Maxim Gekk e3de6ab30d [SPARK-24068] Propagating DataFrameReader's options to Text datasource on schema inferring
## What changes were proposed in this pull request?

While reading CSV or JSON files, DataFrameReader's options are converted to Hadoop's parameters, for example there:
https://github.com/apache/spark/blob/branch-2.3/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala#L302

but the options are not propagated to Text datasource on schema inferring, for instance:
https://github.com/apache/spark/blob/branch-2.3/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala#L184-L188

The PR proposes propagation of user's options to Text datasource on scheme inferring in similar way as user's options are converted to Hadoop parameters if schema is specified.

## How was this patch tested?
The changes were tested manually by using https://github.com/twitter/hadoop-lzo:

```
hadoop-lzo> mvn clean package
hadoop-lzo> ln -s ./target/hadoop-lzo-0.4.21-SNAPSHOT.jar ./hadoop-lzo.jar
```
Create 2 test files in JSON and CSV format and compress them:
```shell
$ cat test.csv
col1|col2
a|1
$ lzop test.csv
$ cat test.json
{"col1":"a","col2":1}
$ lzop test.json
```
Run `spark-shell` with hadoop-lzo:
```
bin/spark-shell --jars ~/hadoop-lzo/hadoop-lzo.jar
```
reading compressed CSV and JSON without schema:
```scala
spark.read.option("io.compression.codecs", "com.hadoop.compression.lzo.LzopCodec").option("inferSchema",true).option("header",true).option("sep","|").csv("test.csv.lzo").show()
+----+----+
|col1|col2|
+----+----+
|   a|   1|
+----+----+
```
```scala
spark.read.option("io.compression.codecs", "com.hadoop.compression.lzo.LzopCodec").option("multiLine", true).json("test.json.lzo").printSchema
root
 |-- col1: string (nullable = true)
 |-- col2: long (nullable = true)
```

Author: Maxim Gekk <maxim.gekk@databricks.com>
Author: Maxim Gekk <max.gekk@gmail.com>

Closes #21182 from MaxGekk/text-options.
2018-05-09 08:32:20 +08:00
Yuming Wang 487faf17ab [SPARK-24117][SQL] Unified the getSizePerRow
## What changes were proposed in this pull request?

This pr unified the `getSizePerRow` because `getSizePerRow` is used in many places. For example:

1. [LocalRelation.scala#L80](f70f46d1e5/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala (L80))
2. [SizeInBytesOnlyStatsPlanVisitor.scala#L36](76b8b840dd/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala (L36))

## How was this patch tested?
Exist tests

Author: Yuming Wang <yumwang@ebay.com>

Closes #21189 from wangyum/SPARK-24117.
2018-05-08 23:43:02 +08:00
gatorsmile 2f6fe7d679 [SPARK-23094][SPARK-23723][SPARK-23724][SQL][FOLLOW-UP] Support custom encoding for json files
## What changes were proposed in this pull request?
This is to add a test case to check the behaviors when users write json in the specified UTF-16/UTF-32 encoding with multiline off.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #21254 from gatorsmile/followupSPARK-23094.
2018-05-08 21:24:35 +08:00
Henry Robinson cd12c5c3ec [SPARK-24128][SQL] Mention configuration option in implicit CROSS JOIN error
## What changes were proposed in this pull request?

Mention `spark.sql.crossJoin.enabled` in error message when an implicit `CROSS JOIN` is detected.

## How was this patch tested?

`CartesianProductSuite` and `JoinSuite`.

Author: Henry Robinson <henry@apache.org>

Closes #21201 from henryr/spark-24128.
2018-05-08 12:21:33 +08:00
Bruce Robbins d83e963724 [SPARK-24043][SQL] Interpreted Predicate should initialize nondeterministic expressions
## What changes were proposed in this pull request?

When creating an InterpretedPredicate instance, initialize any Nondeterministic expressions in the expression tree to avoid java.lang.IllegalArgumentException on later call to eval().

## How was this patch tested?

- sbt SQL tests
- python SQL tests
- new unit test

Author: Bruce Robbins <bersprockets@gmail.com>

Closes #21144 from bersprockets/interpretedpredicate.
2018-05-07 17:54:39 +02:00
Herman van Hovell 4e861db5f1 [SPARK-16406][SQL] Improve performance of LogicalPlan.resolve
## What changes were proposed in this pull request?

`LogicalPlan.resolve(...)` uses linear searches to find an attribute matching a name. This is fine in normal cases, but gets problematic when you try to resolve a large number of columns on a plan with a large number of attributes.

This PR adds an indexing structure to `resolve(...)` in order to find potential matches quicker. This PR improves the reference resolution time for the following code by 4x (11.8s -> 2.4s):

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

spark.time(sql(query))
```
## How was this patch tested?

Existing tests.

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

Closes #14083 from hvanhovell/SPARK-16406.
2018-05-07 11:21:22 +02:00
Marco Gaido e35ad3cadd [SPARK-23930][SQL] Add slice function
## What changes were proposed in this pull request?

The PR add the `slice` function. The behavior of the function is based on Presto's one.

The function slices an array according to the requested start index and length.

## How was this patch tested?

added UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21040 from mgaido91/SPARK-23930.
2018-05-07 16:57:37 +09:00
Kazuaki Ishizaki 7564a9a706 [SPARK-23921][SQL] Add array_sort function
## What changes were proposed in this pull request?

The PR adds the SQL function `array_sort`. The behavior of the function is based on Presto's one.

The function sorts the input array in ascending order. The elements of the input array must be orderable. Null elements will be placed at the end of the returned array.

## How was this patch tested?

Added UTs

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

Closes #21021 from kiszk/SPARK-23921.
2018-05-07 15:22:23 +09:00
gatorsmile f38ea00e83 [SPARK-24017][SQL] Refactor ExternalCatalog to be an interface
## What changes were proposed in this pull request?
This refactors the external catalog to be an interface. It can be easier for the future work in the catalog federation. After the refactoring, `ExternalCatalog` is much cleaner without mixing the listener event generation logic.

## How was this patch tested?
The existing tests

Author: gatorsmile <gatorsmile@gmail.com>

Closes #21122 from gatorsmile/refactorExternalCatalog.
2018-05-06 20:41:32 -07:00
Tathagata Das 47b5b68528 [SPARK-24157][SS] Enabled no-data batches in MicroBatchExecution for streaming aggregation and deduplication.
## What changes were proposed in this pull request?

This PR enables the MicroBatchExecution to run no-data batches if some SparkPlan requires running another batch to output results based on updated watermark / processing time. In this PR, I have enabled streaming aggregations and streaming deduplicates to automatically run addition batch even if new data is available. See https://issues.apache.org/jira/browse/SPARK-24156 for more context.

Major changes/refactoring done in this PR.
- Refactoring MicroBatchExecution - A major point of confusion in MicroBatchExecution control flow was always (at least to me) was that `populateStartOffsets` internally called `constructNextBatch` which was not obvious from just the name "populateStartOffsets" and made the control flow from the main trigger execution loop very confusing (main loop in `runActivatedStream` called `constructNextBatch` but only if `populateStartOffsets` hadn't already called it). Instead, the refactoring makes it cleaner.
    - `populateStartOffsets` only the updates `availableOffsets` and `committedOffsets`. Does not call `constructNextBatch`.
    - Main loop in `runActivatedStream` calls `constructNextBatch` which returns true or false reflecting whether the next batch is ready for executing. This method is now idempotent; if a batch has already been constructed, then it will always return true until the batch has been executed.
    - If next batch is ready then we call `runBatch` or sleep.
    - That's it.

- Refactoring watermark management logic - This has been refactored out from `MicroBatchExecution` in a separate class to simplify `MicroBatchExecution`.

- New method `shouldRunAnotherBatch` in `IncrementalExecution` - This returns true if there is any stateful operation in the last execution plan that requires another batch for state cleanup, etc. This is used to decide whether to construct a batch or not in `constructNextBatch`.

- Changes to stream testing framework - Many tests used CheckLastBatch to validate answers. This assumed that there will be no more batches after the last set of input has been processed, so the last batch is the one that has output corresponding to the last input. This is not true anymore. To account for that, I made two changes.
    - `CheckNewAnswer` is a new test action that verifies the new rows generated since the last time the answer was checked by `CheckAnswer`, `CheckNewAnswer` or `CheckLastBatch`. This is agnostic to how many batches occurred between the last check and now. To do make this easier, I added a common trait between MemorySink and MemorySinkV2 to abstract out some common methods.
    - `assertNumStateRows` has been updated in the same way to be agnostic to batches while checking what the total rows and how many state rows were updated (sums up updates since the last check).

## How was this patch tested?
- Changes made to existing tests - Tests have been changed in one of the following patterns.
    - Tests where the last input was given again to force another batch to be executed and state cleaned up / output generated, they were simplified by removing the extra input.
    - Tests using aggregation+watermark where CheckLastBatch were replaced with CheckNewAnswer to make them batch agnostic.
- New tests added to check whether the flag works for streaming aggregation and deduplication

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

Closes #21220 from tdas/SPARK-24157.
2018-05-04 16:35:24 -07:00
maryannxue e3201e165e [SPARK-24035][SQL] SQL syntax for Pivot
## What changes were proposed in this pull request?

Add SQL support for Pivot according to Pivot grammar defined by Oracle (https://docs.oracle.com/database/121/SQLRF/img_text/pivot_clause.htm) with some simplifications, based on our existing functionality and limitations for Pivot at the backend:
1. For pivot_for_clause (https://docs.oracle.com/database/121/SQLRF/img_text/pivot_for_clause.htm), the column list form is not supported, which means the pivot column can only be one single column.
2. For pivot_in_clause (https://docs.oracle.com/database/121/SQLRF/img_text/pivot_in_clause.htm), the sub-query form and "ANY" is not supported (this is only supported by Oracle for XML anyway).
3. For pivot_in_clause, aliases for the constant values are not supported.

The code changes are:
1. Add parser support for Pivot. Note that according to https://docs.oracle.com/database/121/SQLRF/statements_10002.htm#i2076542, Pivot cannot be used together with lateral views in the from clause. This restriction has been implemented in the Parser rule.
2. Infer group-by expressions: group-by expressions are not explicitly specified in SQL Pivot clause and need to be deduced based on this rule: https://docs.oracle.com/database/121/SQLRF/statements_10002.htm#CHDFAFIE, so we have to post-fix it at query analysis stage.
3. Override Pivot.resolved as "false": for the reason mentioned in [2] and the fact that output attributes change after Pivot being replaced by Project or Aggregate, we avoid resolving parent references until after Pivot has been resolved and replaced.
4. Verify aggregate expressions: only aggregate expressions with or without aliases can appear in the first part of the Pivot clause, and this check is performed as analysis stage.

## How was this patch tested?

A new test suite PivotSuite is added.

Author: maryannxue <maryann.xue@gmail.com>

Closes #21187 from maryannxue/spark-24035.
2018-05-03 17:05:02 -07:00
Wenchen Fan 96a50016bb [SPARK-24169][SQL] JsonToStructs should not access SQLConf at executor side
## What changes were proposed in this pull request?

This PR is extracted from #21190 , to make it easier to backport.

`JsonToStructs` can be serialized to executors and evaluate, we should not call `SQLConf.get.getConf(SQLConf.FROM_JSON_FORCE_NULLABLE_SCHEMA)` in the body.

## How was this patch tested?

tested in #21190

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21226 from cloud-fan/minor4.
2018-05-03 23:36:09 +08:00
Wenchen Fan 417ad92502 [SPARK-23715][SQL] the input of to/from_utc_timestamp can not have timezone
## What changes were proposed in this pull request?

`from_utc_timestamp` assumes its input is in UTC timezone and shifts it to the specified timezone. When the timestamp contains timezone(e.g. `2018-03-13T06:18:23+00:00`), Spark breaks the semantic and respect the timezone in the string. This is not what user expects and the result is different from Hive/Impala. `to_utc_timestamp` has the same problem.

More details please refer to the JIRA ticket.

This PR fixes this by returning null if the input timestamp contains timezone.

## How was this patch tested?

new tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21169 from cloud-fan/from_utc_timezone.
2018-05-03 19:27:01 +08:00
Kazuaki Ishizaki 5be8aab144 [SPARK-23923][SQL] Add cardinality function
## What changes were proposed in this pull request?

The PR adds the SQL function `cardinality`. The behavior of the function is based on Presto's one.

The function returns the length of the array or map stored in the column as `int` while the Presto version returns the value as `BigInt` (`long` in Spark). The discussions regarding the difference of return type are [here](https://github.com/apache/spark/pull/21031#issuecomment-381284638) and [there](https://github.com/apache/spark/pull/21031#discussion_r181622107).

## How was this patch tested?

Added UTs

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

Closes #21031 from kiszk/SPARK-23923.
2018-05-02 13:53:10 -07:00
Marco Gaido 504c9cfd21 [SPARK-24123][SQL] Fix precision issues in monthsBetween with more than 8 digits
## What changes were proposed in this pull request?

SPARK-23902 introduced the ability to retrieve more than 8 digits in `monthsBetween`. Unfortunately, current implementation can cause precision loss in such a case. This was causing also a flaky UT.

This PR mirrors Hive's implementation in order to avoid precision loss also when more than 8 digits are returned.

## How was this patch tested?

running 10000000 times the flaky UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21196 from mgaido91/SPARK-24123.
2018-05-02 13:49:15 -07:00
Marco Gaido 8dbf56c055 [SPARK-24013][SQL] Remove unneeded compress in ApproximatePercentile
## What changes were proposed in this pull request?

`ApproximatePercentile` contains a workaround logic to compress the samples since at the beginning `QuantileSummaries` was ignoring the compression threshold. This problem was fixed in SPARK-17439, but the workaround logic was not removed. So we are compressing the samples many more times than needed: this could lead to critical performance degradation.

This can create serious performance issues in queries like:
```
select approx_percentile(id, array(0.1)) from range(10000000)
```

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21133 from mgaido91/SPARK-24013.
2018-05-02 11:58:55 -07:00
wangyanlin01 7bbec0dced [SPARK-24061][SS] Add TypedFilter support for continuous processing
## What changes were proposed in this pull request?

Add TypedFilter support for continuous processing application.

## How was this patch tested?

unit tests

Author: wangyanlin01 <wangyanlin01@baidu.com>

Closes #21136 from yanlin-Lynn/SPARK-24061.
2018-05-01 16:22:52 +08:00
Maxim Gekk bd14da6fd5 [SPARK-23094][SPARK-23723][SPARK-23724][SQL] Support custom encoding for json files
## What changes were proposed in this pull request?

I propose new option for JSON datasource which allows to specify encoding (charset) of input and output files. Here is an example of using of the option:

```
spark.read.schema(schema)
  .option("multiline", "true")
  .option("encoding", "UTF-16LE")
  .json(fileName)
```

If the option is not specified, charset auto-detection mechanism is used by default.

The option can be used for saving datasets to jsons. Currently Spark is able to save datasets into json files in `UTF-8` charset only. The changes allow to save data in any supported charset. Here is the approximate list of supported charsets by Oracle Java SE: https://docs.oracle.com/javase/8/docs/technotes/guides/intl/encoding.doc.html . An user can specify the charset of output jsons via the charset option like `.option("charset", "UTF-16BE")`. By default the output charset is still `UTF-8` to keep backward compatibility.

The solution has the following restrictions for per-line mode (`multiline = false`):

- If charset is different from UTF-8, the lineSep option must be specified. The option required because Hadoop LineReader cannot detect the line separator correctly. Here is the ticket for solving the issue: https://issues.apache.org/jira/browse/SPARK-23725

- Encoding with [BOM](https://en.wikipedia.org/wiki/Byte_order_mark) are not supported. For example, the `UTF-16` and `UTF-32` encodings are blacklisted. The problem can be solved by https://github.com/MaxGekk/spark-1/pull/2

## How was this patch tested?

I added the following tests:
- reads an json file in `UTF-16LE` encoding with BOM in `multiline` mode
- read json file by using charset auto detection (`UTF-32BE` with BOM)
- read json file using of user's charset (`UTF-16LE`)
- saving in `UTF-32BE` and read the result by standard library (not by Spark)
- checking that default charset is `UTF-8`
- handling wrong (unsupported) charset

Author: Maxim Gekk <maxim.gekk@databricks.com>
Author: Maxim Gekk <max.gekk@gmail.com>

Closes #20937 from MaxGekk/json-encoding-line-sep.
2018-04-29 11:25:31 +08:00
Marco Gaido ad94e8592b [SPARK-23736][SQL][FOLLOWUP] Error message should contains SQL types
## What changes were proposed in this pull request?

In the error messages we should return the SQL types (like `string` rather than the internal types like `StringType`).

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21181 from mgaido91/SPARK-23736_followup.
2018-04-28 10:47:43 +08:00
gatorsmile ce2f919f8d [SPARK-23799][SQL][FOLLOW-UP] FilterEstimation.evaluateInSet produces wrong stats for STRING
## What changes were proposed in this pull request?
`colStat.min` AND `colStat.max` are empty for string type. Thus, `evaluateInSet` should not return zero when either `colStat.min` or `colStat.max`.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #21147 from gatorsmile/cached.
2018-04-26 19:07:13 +08:00
Marco Gaido cd10f9df82 [SPARK-23916][SQL] Add array_join function
## What changes were proposed in this pull request?

The PR adds the SQL function `array_join`. The behavior of the function is based on Presto's one.

The function accepts an `array` of `string` which is to be joined, a `string` which is the delimiter to use between the items of the first argument and optionally a `string` which is used to replace `null` values.

## How was this patch tested?

added UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21011 from mgaido91/SPARK-23916.
2018-04-26 13:37:13 +09:00
Marco Gaido 58c55cb4a6 [SPARK-23902][SQL] Add roundOff flag to months_between
## What changes were proposed in this pull request?

HIVE-15511 introduced the `roundOff` flag in order to disable the rounding to 8 digits which is performed in `months_between`. Since this can be a computational intensive operation, skipping it may improve performances when the rounding is not needed.

## How was this patch tested?

modified existing UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21008 from mgaido91/SPARK-23902.
2018-04-26 12:19:20 +09:00
Wenchen Fan ac4ca7c4dd [SPARK-24012][SQL][TEST][FOLLOWUP] add unit test
## What changes were proposed in this pull request?

a followup of https://github.com/apache/spark/pull/21100

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21154 from cloud-fan/test.
2018-04-25 13:42:44 -07:00
liutang123 64e8408e6f [SPARK-24012][SQL] Union of map and other compatible column
## What changes were proposed in this pull request?
Union of map and other compatible column result in unresolved operator 'Union; exception

Reproduction
`spark-sql>select map(1,2), 'str' union all select map(1,2,3,null), 1`
Output:
```
Error in query: unresolved operator 'Union;;
'Union
:- Project [map(1, 2) AS map(1, 2)#106, str AS str#107]
:  +- OneRowRelation$
+- Project [map(1, cast(2 as int), 3, cast(null as int)) AS map(1, CAST(2 AS INT), 3, CAST(NULL AS INT))#109, 1 AS 1#108]
   +- OneRowRelation$
```
So, we should cast part of columns to be compatible when appropriate.

## How was this patch tested?
Added a test (query union of map and other columns) to SQLQueryTestSuite's union.sql.

Author: liutang123 <liutang123@yeah.net>

Closes #21100 from liutang123/SPARK-24012.
2018-04-25 18:10:51 +08:00
mn-mikke 5fea17b3be [SPARK-23821][SQL] Collection function: flatten
## What changes were proposed in this pull request?

This PR adds a new collection function that transforms an array of arrays into a single array. The PR comprises:
- An expression for flattening array structure
- Flatten function
- A wrapper for PySpark

## How was this patch tested?

New tests added into:
- CollectionExpressionsSuite
- DataFrameFunctionsSuite

## Codegen examples
### Primitive type
```
val df = Seq(
  Seq(Seq(1, 2), Seq(4, 5)),
  Seq(null, Seq(1))
).toDF("i")
df.filter($"i".isNotNull || $"i".isNull).select(flatten($"i")).debugCodegen
```
Result:
```
/* 033 */         boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 034 */         ArrayData inputadapter_value = inputadapter_isNull ?
/* 035 */         null : (inputadapter_row.getArray(0));
/* 036 */
/* 037 */         boolean filter_value = true;
/* 038 */
/* 039 */         if (!(!inputadapter_isNull)) {
/* 040 */           filter_value = inputadapter_isNull;
/* 041 */         }
/* 042 */         if (!filter_value) continue;
/* 043 */
/* 044 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 045 */
/* 046 */         boolean project_isNull = inputadapter_isNull;
/* 047 */         ArrayData project_value = null;
/* 048 */
/* 049 */         if (!inputadapter_isNull) {
/* 050 */           for (int z = 0; !project_isNull && z < inputadapter_value.numElements(); z++) {
/* 051 */             project_isNull |= inputadapter_value.isNullAt(z);
/* 052 */           }
/* 053 */           if (!project_isNull) {
/* 054 */             long project_numElements = 0;
/* 055 */             for (int z = 0; z < inputadapter_value.numElements(); z++) {
/* 056 */               project_numElements += inputadapter_value.getArray(z).numElements();
/* 057 */             }
/* 058 */             if (project_numElements > 2147483632) {
/* 059 */               throw new RuntimeException("Unsuccessful try to flatten an array of arrays with " +
/* 060 */                 project_numElements + " elements due to exceeding the array size limit 2147483632.");
/* 061 */             }
/* 062 */
/* 063 */             long project_size = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(
/* 064 */               project_numElements,
/* 065 */               4);
/* 066 */             if (project_size > 2147483632) {
/* 067 */               throw new RuntimeException("Unsuccessful try to flatten an array of arrays with " +
/* 068 */                 project_size + " bytes of data due to exceeding the limit 2147483632" +
/* 069 */                 " bytes for UnsafeArrayData.");
/* 070 */             }
/* 071 */
/* 072 */             byte[] project_array = new byte[(int)project_size];
/* 073 */             UnsafeArrayData project_tempArrayData = new UnsafeArrayData();
/* 074 */             Platform.putLong(project_array, 16, project_numElements);
/* 075 */             project_tempArrayData.pointTo(project_array, 16, (int)project_size);
/* 076 */             int project_counter = 0;
/* 077 */             for (int k = 0; k < inputadapter_value.numElements(); k++) {
/* 078 */               ArrayData arr = inputadapter_value.getArray(k);
/* 079 */               for (int l = 0; l < arr.numElements(); l++) {
/* 080 */                 if (arr.isNullAt(l)) {
/* 081 */                   project_tempArrayData.setNullAt(project_counter);
/* 082 */                 } else {
/* 083 */                   project_tempArrayData.setInt(
/* 084 */                     project_counter,
/* 085 */                     arr.getInt(l)
/* 086 */                   );
/* 087 */                 }
/* 088 */                 project_counter++;
/* 089 */               }
/* 090 */             }
/* 091 */             project_value = project_tempArrayData;
/* 092 */
/* 093 */           }
/* 094 */
/* 095 */         }
```
### Non-primitive type
```
val df = Seq(
  Seq(Seq("a", "b"), Seq(null, "d")),
  Seq(null, Seq("a"))
).toDF("s")
df.filter($"s".isNotNull || $"s".isNull).select(flatten($"s")).debugCodegen
```
Result:
```
/* 033 */         boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 034 */         ArrayData inputadapter_value = inputadapter_isNull ?
/* 035 */         null : (inputadapter_row.getArray(0));
/* 036 */
/* 037 */         boolean filter_value = true;
/* 038 */
/* 039 */         if (!(!inputadapter_isNull)) {
/* 040 */           filter_value = inputadapter_isNull;
/* 041 */         }
/* 042 */         if (!filter_value) continue;
/* 043 */
/* 044 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 045 */
/* 046 */         boolean project_isNull = inputadapter_isNull;
/* 047 */         ArrayData project_value = null;
/* 048 */
/* 049 */         if (!inputadapter_isNull) {
/* 050 */           for (int z = 0; !project_isNull && z < inputadapter_value.numElements(); z++) {
/* 051 */             project_isNull |= inputadapter_value.isNullAt(z);
/* 052 */           }
/* 053 */           if (!project_isNull) {
/* 054 */             long project_numElements = 0;
/* 055 */             for (int z = 0; z < inputadapter_value.numElements(); z++) {
/* 056 */               project_numElements += inputadapter_value.getArray(z).numElements();
/* 057 */             }
/* 058 */             if (project_numElements > 2147483632) {
/* 059 */               throw new RuntimeException("Unsuccessful try to flatten an array of arrays with " +
/* 060 */                 project_numElements + " elements due to exceeding the array size limit 2147483632.");
/* 061 */             }
/* 062 */
/* 063 */             Object[] project_arrayObject = new Object[(int)project_numElements];
/* 064 */             int project_counter = 0;
/* 065 */             for (int k = 0; k < inputadapter_value.numElements(); k++) {
/* 066 */               ArrayData arr = inputadapter_value.getArray(k);
/* 067 */               for (int l = 0; l < arr.numElements(); l++) {
/* 068 */                 project_arrayObject[project_counter] = arr.getUTF8String(l);
/* 069 */                 project_counter++;
/* 070 */               }
/* 071 */             }
/* 072 */             project_value = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_arrayObject);
/* 073 */
/* 074 */           }
/* 075 */
/* 076 */         }
```

Author: mn-mikke <mrkAha12346github>

Closes #20938 from mn-mikke/feature/array-api-flatten-to-master.
2018-04-25 11:19:08 +09:00
Takeshi Yamamuro 4926a7c2f0 [SPARK-23589][SQL][FOLLOW-UP] Reuse InternalRow in ExternalMapToCatalyst eval
## What changes were proposed in this pull request?
This pr is a follow-up of #20980 and fixes code to reuse `InternalRow` for converting input keys/values in `ExternalMapToCatalyst` eval.

## How was this patch tested?
Existing tests.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21137 from maropu/SPARK-23589-FOLLOWUP.
2018-04-24 17:52:05 +02:00
Marco Gaido 281c1ca0dc [SPARK-23973][SQL] Remove consecutive Sorts
## What changes were proposed in this pull request?

In SPARK-23375 we introduced the ability of removing `Sort` operation during query optimization if the data is already sorted. In this follow-up we remove also a `Sort` which is followed by another `Sort`: in this case the first sort is not needed and can be safely removed.

The PR starts from henryr's comment: https://github.com/apache/spark/pull/20560#discussion_r180601594. So credit should be given to him.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21072 from mgaido91/SPARK-23973.
2018-04-24 10:11:09 +08:00
Takeshi Yamamuro afbdf42730 [SPARK-23589][SQL] ExternalMapToCatalyst should support interpreted execution
## What changes were proposed in this pull request?
This pr supported interpreted mode for `ExternalMapToCatalyst`.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20980 from maropu/SPARK-23589.
2018-04-23 14:28:28 +02:00
Wenchen Fan d87d30e4fe [SPARK-23564][SQL] infer additional filters from constraints for join's children
## What changes were proposed in this pull request?

The existing query constraints framework has 2 steps:
1. propagate constraints bottom up.
2. use constraints to infer additional filters for better data pruning.

For step 2, it mostly helps with Join, because we can connect the constraints from children to the join condition and infer powerful filters to prune the data of the join sides. e.g., the left side has constraints `a = 1`, the join condition is `left.a = right.a`, then we can infer `right.a = 1` to the right side and prune the right side a lot.

However, the current logic of inferring filters from constraints for Join is pretty weak. It infers the filters from Join's constraints. Some joins like left semi/anti exclude output from right side and the right side constraints will be lost here.

This PR propose to check the left and right constraints individually, expand the constraints with join condition and add filters to children of join directly, instead of adding to the join condition.

This reverts https://github.com/apache/spark/pull/20670 , covers https://github.com/apache/spark/pull/20717 and https://github.com/apache/spark/pull/20816

This is inspired by the original PRs and the tests are all from these PRs. Thanks to the authors mgaido91 maryannxue KaiXinXiaoLei !

## How was this patch tested?

new tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21083 from cloud-fan/join.
2018-04-23 20:21:01 +08:00
Wenchen Fan f70f46d1e5 [SPARK-23877][SQL][FOLLOWUP] use PhysicalOperation to simplify the handling of Project and Filter over partitioned relation
## What changes were proposed in this pull request?

A followup of https://github.com/apache/spark/pull/20988

`PhysicalOperation` can collect Project and Filters over a certain plan and substitute the alias with the original attributes in the bottom plan. We can use it in `OptimizeMetadataOnlyQuery` rule to handle the Project and Filter over partitioned relation.

## How was this patch tested?

existing test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21111 from cloud-fan/refactor.
2018-04-23 20:18:50 +08:00
Mykhailo Shtelma c48085aa91 [SPARK-23799][SQL] FilterEstimation.evaluateInSet produces devision by zero in a case of empty table with analyzed statistics
>What changes were proposed in this pull request?

During evaluation of IN conditions, if the source data frame, is represented by a plan, that uses hive table with columns, which were previously analysed, and the plan has conditions for these fields, that cannot be satisfied (which leads us to an empty data frame), FilterEstimation.evaluateInSet method produces NumberFormatException and ClassCastException.
In order to fix this bug, method FilterEstimation.evaluateInSet at first checks, if distinct count is not zero, and also checks if colStat.min and colStat.max  are defined, and only in this case proceeds with the calculation. If at least one of the conditions is not satisfied, zero is returned.

>How was this patch tested?

In order to test the PR two tests were implemented: one in FilterEstimationSuite, that tests the plan with the statistics that violates the conditions mentioned above,  and another one in StatisticsCollectionSuite, that test the whole process of analysis/optimisation of the query, that leads to the problems, mentioned in the first section.

Author: Mykhailo Shtelma <mykhailo.shtelma@bearingpoint.com>
Author: smikesh <mshtelma@gmail.com>

Closes #21052 from mshtelma/filter_estimation_evaluateInSet_Bugs.
2018-04-21 23:33:57 -07:00
gatorsmile 7bc853d089 [SPARK-24033][SQL] Fix Mismatched of Window Frame specifiedwindowframe(RowFrame, -1, -1)
## What changes were proposed in this pull request?

When the OffsetWindowFunction's frame is `UnaryMinus(Literal(1))` but the specified window frame has been simplified to `Literal(-1)` by some optimizer rules e.g., `ConstantFolding`. Thus, they do not match and cause the following error:
```
org.apache.spark.sql.AnalysisException: Window Frame specifiedwindowframe(RowFrame, -1, -1) must match the required frame specifiedwindowframe(RowFrame, -1, -1);
at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.failAnalysis(CheckAnalysis.scala:41)
at org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:91)
at
```
## How was this patch tested?
Added a test

Author: gatorsmile <gatorsmile@gmail.com>

Closes #21115 from gatorsmile/fixLag.
2018-04-21 10:45:12 -07:00
Takeshi Yamamuro 0dd97f6ea4 [SPARK-23595][SQL] ValidateExternalType should support interpreted execution
## What changes were proposed in this pull request?
This pr supported interpreted mode for `ValidateExternalType`.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20757 from maropu/SPARK-23595.
2018-04-20 15:02:27 +02:00
Takeshi Yamamuro 074a7f9053 [SPARK-23588][SQL][FOLLOW-UP] Resolve a map builder method per execution in CatalystToExternalMap
## What changes were proposed in this pull request?
This pr is a follow-up pr of #20979 and fixes code to resolve a map builder method per execution instead of per row in `CatalystToExternalMap`.

## How was this patch tested?
Existing tests.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21112 from maropu/SPARK-23588-FOLLOWUP.
2018-04-20 14:43:47 +02:00
mn-mikke e6b466084c [SPARK-23736][SQL] Extending the concat function to support array columns
## What changes were proposed in this pull request?
The PR adds a logic for easy concatenation of multiple array columns and covers:
- Concat expression has been extended to support array columns
- A Python wrapper

## How was this patch tested?
New tests added into:
- CollectionExpressionsSuite
- DataFrameFunctionsSuite
- typeCoercion/native/concat.sql

## Codegen examples
### Primitive-type elements
```
val df = Seq(
  (Seq(1 ,2), Seq(3, 4)),
  (Seq(1, 2, 3), null)
).toDF("a", "b")
df.filter('a.isNotNull).select(concat('a, 'b)).debugCodegen()
```
Result:
```
/* 033 */         boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 034 */         ArrayData inputadapter_value = inputadapter_isNull ?
/* 035 */         null : (inputadapter_row.getArray(0));
/* 036 */
/* 037 */         if (!(!inputadapter_isNull)) continue;
/* 038 */
/* 039 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 040 */
/* 041 */         ArrayData[] project_args = new ArrayData[2];
/* 042 */
/* 043 */         if (!false) {
/* 044 */           project_args[0] = inputadapter_value;
/* 045 */         }
/* 046 */
/* 047 */         boolean inputadapter_isNull1 = inputadapter_row.isNullAt(1);
/* 048 */         ArrayData inputadapter_value1 = inputadapter_isNull1 ?
/* 049 */         null : (inputadapter_row.getArray(1));
/* 050 */         if (!inputadapter_isNull1) {
/* 051 */           project_args[1] = inputadapter_value1;
/* 052 */         }
/* 053 */
/* 054 */         ArrayData project_value = new Object() {
/* 055 */           public ArrayData concat(ArrayData[] args) {
/* 056 */             for (int z = 0; z < 2; z++) {
/* 057 */               if (args[z] == null) return null;
/* 058 */             }
/* 059 */
/* 060 */             long project_numElements = 0L;
/* 061 */             for (int z = 0; z < 2; z++) {
/* 062 */               project_numElements += args[z].numElements();
/* 063 */             }
/* 064 */             if (project_numElements > 2147483632) {
/* 065 */               throw new RuntimeException("Unsuccessful try to concat arrays with " + project_numElements +
/* 066 */                 " elements due to exceeding the array size limit 2147483632.");
/* 067 */             }
/* 068 */
/* 069 */             long project_size = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(
/* 070 */               project_numElements,
/* 071 */               4);
/* 072 */             if (project_size > 2147483632) {
/* 073 */               throw new RuntimeException("Unsuccessful try to concat arrays with " + project_size +
/* 074 */                 " bytes of data due to exceeding the limit 2147483632 bytes" +
/* 075 */                 " for UnsafeArrayData.");
/* 076 */             }
/* 077 */
/* 078 */             byte[] project_array = new byte[(int)project_size];
/* 079 */             UnsafeArrayData project_arrayData = new UnsafeArrayData();
/* 080 */             Platform.putLong(project_array, 16, project_numElements);
/* 081 */             project_arrayData.pointTo(project_array, 16, (int)project_size);
/* 082 */             int project_counter = 0;
/* 083 */             for (int y = 0; y < 2; y++) {
/* 084 */               for (int z = 0; z < args[y].numElements(); z++) {
/* 085 */                 if (args[y].isNullAt(z)) {
/* 086 */                   project_arrayData.setNullAt(project_counter);
/* 087 */                 } else {
/* 088 */                   project_arrayData.setInt(
/* 089 */                     project_counter,
/* 090 */                     args[y].getInt(z)
/* 091 */                   );
/* 092 */                 }
/* 093 */                 project_counter++;
/* 094 */               }
/* 095 */             }
/* 096 */             return project_arrayData;
/* 097 */           }
/* 098 */         }.concat(project_args);
/* 099 */         boolean project_isNull = project_value == null;
```

### Non-primitive-type elements
```
val df = Seq(
  (Seq("aa" ,"bb"), Seq("ccc", "ddd")),
  (Seq("x", "y"), null)
).toDF("a", "b")
df.filter('a.isNotNull).select(concat('a, 'b)).debugCodegen()
```
Result:
```
/* 033 */         boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 034 */         ArrayData inputadapter_value = inputadapter_isNull ?
/* 035 */         null : (inputadapter_row.getArray(0));
/* 036 */
/* 037 */         if (!(!inputadapter_isNull)) continue;
/* 038 */
/* 039 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 040 */
/* 041 */         ArrayData[] project_args = new ArrayData[2];
/* 042 */
/* 043 */         if (!false) {
/* 044 */           project_args[0] = inputadapter_value;
/* 045 */         }
/* 046 */
/* 047 */         boolean inputadapter_isNull1 = inputadapter_row.isNullAt(1);
/* 048 */         ArrayData inputadapter_value1 = inputadapter_isNull1 ?
/* 049 */         null : (inputadapter_row.getArray(1));
/* 050 */         if (!inputadapter_isNull1) {
/* 051 */           project_args[1] = inputadapter_value1;
/* 052 */         }
/* 053 */
/* 054 */         ArrayData project_value = new Object() {
/* 055 */           public ArrayData concat(ArrayData[] args) {
/* 056 */             for (int z = 0; z < 2; z++) {
/* 057 */               if (args[z] == null) return null;
/* 058 */             }
/* 059 */
/* 060 */             long project_numElements = 0L;
/* 061 */             for (int z = 0; z < 2; z++) {
/* 062 */               project_numElements += args[z].numElements();
/* 063 */             }
/* 064 */             if (project_numElements > 2147483632) {
/* 065 */               throw new RuntimeException("Unsuccessful try to concat arrays with " + project_numElements +
/* 066 */                 " elements due to exceeding the array size limit 2147483632.");
/* 067 */             }
/* 068 */
/* 069 */             Object[] project_arrayObjects = new Object[(int)project_numElements];
/* 070 */             int project_counter = 0;
/* 071 */             for (int y = 0; y < 2; y++) {
/* 072 */               for (int z = 0; z < args[y].numElements(); z++) {
/* 073 */                 project_arrayObjects[project_counter] = args[y].getUTF8String(z);
/* 074 */                 project_counter++;
/* 075 */               }
/* 076 */             }
/* 077 */             return new org.apache.spark.sql.catalyst.util.GenericArrayData(project_arrayObjects);
/* 078 */           }
/* 079 */         }.concat(project_args);
/* 080 */         boolean project_isNull = project_value == null;
```

Author: mn-mikke <mrkAha12346github>

Closes #20858 from mn-mikke/feature/array-api-concat_arrays-to-master.
2018-04-20 14:58:11 +09:00
Xingbo Jiang d96c3e33cc [SPARK-21811][SQL] Fix the inconsistency behavior when finding the widest common type
## What changes were proposed in this pull request?

Currently we find the wider common type by comparing the two types from left to right, this can be a problem when you have two data types which don't have a common type but each can be promoted to StringType.

For instance, if you have a table with the schema:
[c1: date, c2: string, c3: int]

The following succeeds:
SELECT coalesce(c1, c2, c3) FROM table

While the following produces an exception:
SELECT coalesce(c1, c3, c2) FROM table

This is only a issue when the seq of dataTypes contains `StringType` and all the types can do string promotion.

close #19033

## How was this patch tested?

Add test in `TypeCoercionSuite`

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

Closes #21074 from jiangxb1987/typeCoercion.
2018-04-19 21:21:22 +08:00
Takeshi Yamamuro e13416502f [SPARK-23588][SQL] CatalystToExternalMap should support interpreted execution
## What changes were proposed in this pull request?
This pr supported interpreted mode for `CatalystToExternalMap`.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20979 from maropu/SPARK-23588.
2018-04-19 14:42:50 +02:00
Takeshi Yamamuro 1b08c4393c [SPARK-23584][SQL] NewInstance should support interpreted execution
## What changes were proposed in this pull request?
This pr supported interpreted mode for `NewInstance`.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20778 from maropu/SPARK-23584.
2018-04-19 14:38:26 +02:00
Kazuaki Ishizaki 46bb2b5129 [SPARK-23924][SQL] Add element_at function
## What changes were proposed in this pull request?

The PR adds the SQL function `element_at`. The behavior of the function is based on Presto's one.

This function returns element of array at given index in value if column is array, or returns value for the given key in value if column is map.

## How was this patch tested?

Added UTs

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

Closes #21053 from kiszk/SPARK-23924.
2018-04-19 21:00:10 +09:00
Kazuaki Ishizaki d5bec48b9c [SPARK-23919][SQL] Add array_position function
## What changes were proposed in this pull request?

The PR adds the SQL function `array_position`. The behavior of the function is based on Presto's one.

The function returns the position of the first occurrence of the element in array x (or 0 if not found) using 1-based index as BigInt.

## How was this patch tested?

Added UTs

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

Closes #21037 from kiszk/SPARK-23919.
2018-04-19 11:59:17 +09:00
Liang-Chi Hsieh a9066478f6 [SPARK-23875][SQL][FOLLOWUP] Add IndexedSeq wrapper for ArrayData
## What changes were proposed in this pull request?

Use specified accessor in `ArrayData.foreach` and `toArray`.

## How was this patch tested?

Existing tests.

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

Closes #21099 from viirya/SPARK-23875-followup.
2018-04-19 00:05:47 +02:00
Takuya UESHIN f09a9e9418 [SPARK-24007][SQL] EqualNullSafe for FloatType and DoubleType might generate a wrong result by codegen.
## What changes were proposed in this pull request?

`EqualNullSafe` for `FloatType` and `DoubleType` might generate a wrong result by codegen.

```scala
scala> val df = Seq((Some(-1.0d), None), (None, Some(-1.0d))).toDF()
df: org.apache.spark.sql.DataFrame = [_1: double, _2: double]

scala> df.show()
+----+----+
|  _1|  _2|
+----+----+
|-1.0|null|
|null|-1.0|
+----+----+

scala> df.filter("_1 <=> _2").show()
+----+----+
|  _1|  _2|
+----+----+
|-1.0|null|
|null|-1.0|
+----+----+
```

The result should be empty but the result remains two rows.

## How was this patch tested?

Added a test.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #21094 from ueshin/issues/SPARK-24007/equalnullsafe.
2018-04-18 08:22:05 -07:00
mn-mikke f81fa478ff [SPARK-23926][SQL] Extending reverse function to support ArrayType arguments
## What changes were proposed in this pull request?

This PR extends `reverse` functions to be able to operate over array columns and covers:
- Introduction of `Reverse` expression that represents logic for reversing arrays and also strings
- Removal of `StringReverse` expression
- A wrapper for PySpark

## How was this patch tested?

New tests added into:
- CollectionExpressionsSuite
- DataFrameFunctionsSuite

## Codegen examples
### Primitive type
```
val df = Seq(
  Seq(1, 3, 4, 2),
  null
).toDF("i")
df.filter($"i".isNotNull || $"i".isNull).select(reverse($"i")).debugCodegen
```
Result:
```
/* 032 */         boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 033 */         ArrayData inputadapter_value = inputadapter_isNull ?
/* 034 */         null : (inputadapter_row.getArray(0));
/* 035 */
/* 036 */         boolean filter_value = true;
/* 037 */
/* 038 */         if (!(!inputadapter_isNull)) {
/* 039 */           filter_value = inputadapter_isNull;
/* 040 */         }
/* 041 */         if (!filter_value) continue;
/* 042 */
/* 043 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 044 */
/* 045 */         boolean project_isNull = inputadapter_isNull;
/* 046 */         ArrayData project_value = null;
/* 047 */
/* 048 */         if (!inputadapter_isNull) {
/* 049 */           final int project_length = inputadapter_value.numElements();
/* 050 */           project_value = inputadapter_value.copy();
/* 051 */           for(int k = 0; k < project_length / 2; k++) {
/* 052 */             int l = project_length - k - 1;
/* 053 */             boolean isNullAtK = project_value.isNullAt(k);
/* 054 */             boolean isNullAtL = project_value.isNullAt(l);
/* 055 */             if(!isNullAtK) {
/* 056 */               int el = project_value.getInt(k);
/* 057 */               if(!isNullAtL) {
/* 058 */                 project_value.setInt(k, project_value.getInt(l));
/* 059 */               } else {
/* 060 */                 project_value.setNullAt(k);
/* 061 */               }
/* 062 */               project_value.setInt(l, el);
/* 063 */             } else if (!isNullAtL) {
/* 064 */               project_value.setInt(k, project_value.getInt(l));
/* 065 */               project_value.setNullAt(l);
/* 066 */             }
/* 067 */           }
/* 068 */
/* 069 */         }
```
### Non-primitive type
```
val df = Seq(
  Seq("a", "c", "d", "b"),
  null
).toDF("s")
df.filter($"s".isNotNull || $"s".isNull).select(reverse($"s")).debugCodegen
```
Result:
```
/* 032 */         boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 033 */         ArrayData inputadapter_value = inputadapter_isNull ?
/* 034 */         null : (inputadapter_row.getArray(0));
/* 035 */
/* 036 */         boolean filter_value = true;
/* 037 */
/* 038 */         if (!(!inputadapter_isNull)) {
/* 039 */           filter_value = inputadapter_isNull;
/* 040 */         }
/* 041 */         if (!filter_value) continue;
/* 042 */
/* 043 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 044 */
/* 045 */         boolean project_isNull = inputadapter_isNull;
/* 046 */         ArrayData project_value = null;
/* 047 */
/* 048 */         if (!inputadapter_isNull) {
/* 049 */           final int project_length = inputadapter_value.numElements();
/* 050 */           project_value = new org.apache.spark.sql.catalyst.util.GenericArrayData(new Object[project_length]);
/* 051 */           for(int k = 0; k < project_length; k++) {
/* 052 */             int l = project_length - k - 1;
/* 053 */             project_value.update(k, inputadapter_value.getUTF8String(l));
/* 054 */           }
/* 055 */
/* 056 */         }
```

Author: mn-mikke <mrkAha12346github>

Closes #21034 from mn-mikke/feature/array-api-reverse-to-master.
2018-04-18 18:41:55 +09:00
maryannxue 1e3b8762a8 [SPARK-21479][SQL] Outer join filter pushdown in null supplying table when condition is on one of the joined columns
## What changes were proposed in this pull request?

Added `TransitPredicateInOuterJoin` optimization rule that transits constraints from the preserved side of an outer join to the null-supplying side. The constraints of the join operator will remain unchanged.

## How was this patch tested?

Added 3 tests in `InferFiltersFromConstraintsSuite`.

Author: maryannxue <maryann.xue@gmail.com>

Closes #20816 from maryannxue/spark-21479.
2018-04-18 10:36:41 +08:00
Marco Gaido f39e82ce15 [SPARK-23986][SQL] freshName can generate non-unique names
## What changes were proposed in this pull request?

We are using `CodegenContext.freshName` to get a unique name for any new variable we are adding. Unfortunately, this method currently fails to create a unique name when we request more than one instance of variables with starting name `name1` and an instance with starting name `name11`.

The PR changes the way a new name is generated by `CodegenContext.freshName` so that we generate unique names in this scenario too.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21080 from mgaido91/SPARK-23986.
2018-04-18 00:35:44 +08:00
jinxing ed4101d29f [SPARK-22676] Avoid iterating all partition paths when spark.sql.hive.verifyPartitionPath=true
## What changes were proposed in this pull request?

In current code, it will scanning all partition paths when spark.sql.hive.verifyPartitionPath=true.
e.g. table like below:
```
CREATE TABLE `test`(
`id` int,
`age` int,
`name` string)
PARTITIONED BY (
`A` string,
`B` string)
load data local inpath '/tmp/data0' into table test partition(A='00', B='00')
load data local inpath '/tmp/data1' into table test partition(A='01', B='01')
load data local inpath '/tmp/data2' into table test partition(A='10', B='10')
load data local inpath '/tmp/data3' into table test partition(A='11', B='11')
```
If I query with SQL – "select * from test where A='00' and B='01'  ", current code will scan all partition paths including '/data/A=00/B=00', '/data/A=00/B=00', '/data/A=01/B=01', '/data/A=10/B=10', '/data/A=11/B=11'. It costs much time and memory cost.

This pr proposes to avoid iterating all partition paths. Add a config `spark.files.ignoreMissingFiles` and ignore the `file not found` when `getPartitions/compute`(for hive table scan). This is much like the logic brought by
`spark.sql.files.ignoreMissingFiles`(which is for datasource scan).

## How was this patch tested?
UT

Author: jinxing <jinxing6042@126.com>

Closes #19868 from jinxing64/SPARK-22676.
2018-04-17 21:52:33 +08:00
Marco Gaido 0a9172a05e [SPARK-23835][SQL] Add not-null check to Tuples' arguments deserialization
## What changes were proposed in this pull request?

There was no check on nullability for arguments of `Tuple`s. This could lead to have weird behavior when a null value had to be deserialized into a non-nullable Scala object: in those cases, the `null` got silently transformed in a valid value (like `-1` for `Int`), corresponding to the default value we are using in the SQL codebase. This situation was very likely to happen when deserializing to a Tuple of primitive Scala types (like Double, Int, ...).

The PR adds the `AssertNotNull` to arguments of tuples which have been asked to be converted to non-nullable types.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20976 from mgaido91/SPARK-23835.
2018-04-17 21:45:20 +08:00
Liang-Chi Hsieh 30ffb53cad [SPARK-23875][SQL] Add IndexedSeq wrapper for ArrayData
## What changes were proposed in this pull request?

We don't have a good way to sequentially access `UnsafeArrayData` with a common interface such as `Seq`. An example is `MapObject` where we need to access several sequence collection types together. But `UnsafeArrayData` doesn't implement `ArrayData.array`. Calling `toArray` will copy the entire array. We can provide an `IndexedSeq` wrapper for `ArrayData`, so we can avoid copying the entire array.

## How was this patch tested?

Added test.

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

Closes #20984 from viirya/SPARK-23875.
2018-04-17 15:09:36 +02:00
Marco Gaido 14844a62c0 [SPARK-23918][SQL] Add array_min function
## What changes were proposed in this pull request?

The PR adds the SQL function `array_min`. It takes an array as argument and returns the minimum value in it.

## How was this patch tested?

added UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21025 from mgaido91/SPARK-23918.
2018-04-17 17:55:35 +09:00
Liang-Chi Hsieh fd990a908b [SPARK-23873][SQL] Use accessors in interpreted LambdaVariable
## What changes were proposed in this pull request?

Currently, interpreted execution of `LambdaVariable` just uses `InternalRow.get` to access element. We should use specified accessors if possible.

## How was this patch tested?

Added test.

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

Closes #20981 from viirya/SPARK-23873.
2018-04-16 22:45:57 +02:00
Marco Gaido 6931022031 [SPARK-23917][SQL] Add array_max function
## What changes were proposed in this pull request?

The PR adds the SQL function `array_max`. It takes an array as argument and returns the maximum value in it.

## How was this patch tested?

added UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21024 from mgaido91/SPARK-23917.
2018-04-15 21:45:55 -07:00
Liang-Chi Hsieh 73f28530d6 [SPARK-23979][SQL] MultiAlias should not be a CodegenFallback
## What changes were proposed in this pull request?

Just found `MultiAlias` is a `CodegenFallback`. It should not be as looks like `MultiAlias` won't be evaluated.

## How was this patch tested?

Existing tests.

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

Closes #21065 from viirya/multialias-without-codegenfallback.
2018-04-14 08:59:04 +08:00
Tathagata Das cbb41a0c5b [SPARK-23966][SS] Refactoring all checkpoint file writing logic in a common CheckpointFileManager interface
## What changes were proposed in this pull request?

Checkpoint files (offset log files, state store files) in Structured Streaming must be written atomically such that no partial files are generated (would break fault-tolerance guarantees). Currently, there are 3 locations which try to do this individually, and in some cases, incorrectly.

1. HDFSOffsetMetadataLog - This uses a FileManager interface to use any implementation of `FileSystem` or `FileContext` APIs. It preferably loads `FileContext` implementation as FileContext of HDFS has atomic renames.
1. HDFSBackedStateStore (aka in-memory state store)
  - Writing a version.delta file - This uses FileSystem APIs only to perform a rename. This is incorrect as rename is not atomic in HDFS FileSystem implementation.
  - Writing a snapshot file - Same as above.

#### Current problems:
1. State Store behavior is incorrect - HDFS FileSystem implementation does not have atomic rename.
1. Inflexible - Some file systems provide mechanisms other than write-to-temp-file-and-rename for writing atomically and more efficiently. For example, with S3 you can write directly to the final file and it will be made visible only when the entire file is written and closed correctly. Any failure can be made to terminate the writing without making any partial files visible in S3. The current code does not abstract out this mechanism enough that it can be customized.

#### Solution:

1. Introduce a common interface that all 3 cases above can use to write checkpoint files atomically.
2. This interface must provide the necessary interfaces that allow customization of the write-and-rename mechanism.

This PR does that by introducing the interface `CheckpointFileManager` and modifying `HDFSMetadataLog` and `HDFSBackedStateStore` to use the interface. Similar to earlier `FileManager`, there are implementations based on `FileSystem` and `FileContext` APIs, and the latter implementation is preferred to make it work correctly with HDFS.

The key method this interface has is `createAtomic(path, overwrite)` which returns a `CancellableFSDataOutputStream` that has the method `cancel()`. All users of this method need to either call `close()` to successfully write the file, or `cancel()` in case of an error.

## How was this patch tested?
New tests in `CheckpointFileManagerSuite` and slightly modified existing tests.

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

Closes #21048 from tdas/SPARK-23966.
2018-04-13 16:31:39 -07:00
Marco Gaido 25892f3cc9 [SPARK-23375][SQL] Eliminate unneeded Sort in Optimizer
## What changes were proposed in this pull request?

Added a new rule to remove Sort operation when its child is already sorted.
For instance, this simple code:
```
spark.sparkContext.parallelize(Seq(("a", "b"))).toDF("a", "b").registerTempTable("table1")
val df = sql(s"""SELECT b
                | FROM (
                |     SELECT a, b
                |     FROM table1
                |     ORDER BY a
                | ) t
                | ORDER BY a""".stripMargin)
df.explain(true)
```
before the PR produces this plan:
```
== Parsed Logical Plan ==
'Sort ['a ASC NULLS FIRST], true
+- 'Project ['b]
   +- 'SubqueryAlias t
      +- 'Sort ['a ASC NULLS FIRST], true
         +- 'Project ['a, 'b]
            +- 'UnresolvedRelation `table1`

== Analyzed Logical Plan ==
b: string
Project [b#7]
+- Sort [a#6 ASC NULLS FIRST], true
   +- Project [b#7, a#6]
      +- SubqueryAlias t
         +- Sort [a#6 ASC NULLS FIRST], true
            +- Project [a#6, b#7]
               +- SubqueryAlias table1
                  +- Project [_1#3 AS a#6, _2#4 AS b#7]
                     +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1, true, false) AS _1#3, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2, true, false) AS _2#4]
                        +- ExternalRDD [obj#2]

== Optimized Logical Plan ==
Project [b#7]
+- Sort [a#6 ASC NULLS FIRST], true
   +- Project [b#7, a#6]
      +- Sort [a#6 ASC NULLS FIRST], true
         +- Project [_1#3 AS a#6, _2#4 AS b#7]
            +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._1, true, false) AS _1#3, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._2, true, false) AS _2#4]
               +- ExternalRDD [obj#2]

== Physical Plan ==
*(3) Project [b#7]
+- *(3) Sort [a#6 ASC NULLS FIRST], true, 0
   +- Exchange rangepartitioning(a#6 ASC NULLS FIRST, 200)
      +- *(2) Project [b#7, a#6]
         +- *(2) Sort [a#6 ASC NULLS FIRST], true, 0
            +- Exchange rangepartitioning(a#6 ASC NULLS FIRST, 200)
               +- *(1) Project [_1#3 AS a#6, _2#4 AS b#7]
                  +- *(1) SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._1, true, false) AS _1#3, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._2, true, false) AS _2#4]
                     +- Scan ExternalRDDScan[obj#2]
```

while after the PR produces:

```
== Parsed Logical Plan ==
'Sort ['a ASC NULLS FIRST], true
+- 'Project ['b]
   +- 'SubqueryAlias t
      +- 'Sort ['a ASC NULLS FIRST], true
         +- 'Project ['a, 'b]
            +- 'UnresolvedRelation `table1`

== Analyzed Logical Plan ==
b: string
Project [b#7]
+- Sort [a#6 ASC NULLS FIRST], true
   +- Project [b#7, a#6]
      +- SubqueryAlias t
         +- Sort [a#6 ASC NULLS FIRST], true
            +- Project [a#6, b#7]
               +- SubqueryAlias table1
                  +- Project [_1#3 AS a#6, _2#4 AS b#7]
                     +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1, true, false) AS _1#3, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2, true, false) AS _2#4]
                        +- ExternalRDD [obj#2]

== Optimized Logical Plan ==
Project [b#7]
+- Sort [a#6 ASC NULLS FIRST], true
   +- Project [_1#3 AS a#6, _2#4 AS b#7]
      +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._1, true, false) AS _1#3, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._2, true, false) AS _2#4]
         +- ExternalRDD [obj#2]

== Physical Plan ==
*(2) Project [b#7]
+- *(2) Sort [a#6 ASC NULLS FIRST], true, 0
   +- Exchange rangepartitioning(a#6 ASC NULLS FIRST, 5)
      +- *(1) Project [_1#3 AS a#6, _2#4 AS b#7]
         +- *(1) SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._1, true, false) AS _1#3, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._2, true, false) AS _2#4]
            +- Scan ExternalRDDScan[obj#2]
```

this means that an unnecessary sort operation is not performed after the PR.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20560 from mgaido91/SPARK-23375.
2018-04-14 01:01:00 +08:00
yucai 0323e61465 [SPARK-23905][SQL] Add UDF weekday
## What changes were proposed in this pull request?

Add UDF weekday

## How was this patch tested?

A new test

Author: yucai <yyu1@ebay.com>

Closes #21009 from yucai/SPARK-23905.
2018-04-13 00:00:04 -07:00
jerryshao 14291b061b [SPARK-23748][SS] Fix SS continuous process doesn't support SubqueryAlias issue
## What changes were proposed in this pull request?

Current SS continuous doesn't support processing on temp table or `df.as("xxx")`, SS will throw an exception as LogicalPlan not supported, details described in [here](https://issues.apache.org/jira/browse/SPARK-23748).

So here propose to add this support.

## How was this patch tested?

new UT.

Author: jerryshao <sshao@hortonworks.com>

Closes #21017 from jerryshao/SPARK-23748.
2018-04-12 20:00:25 -07:00
Kazuaki Ishizaki 0b19122d43 [SPARK-23762][SQL] UTF8StringBuffer uses MemoryBlock
## What changes were proposed in this pull request?

This PR tries to use `MemoryBlock` in `UTF8StringBuffer`. In general, there are two advantages to use `MemoryBlock`.

1. Has clean API calls rather than using a Java array or `PlatformMemory`
2. Improve runtime performance of memory access instead of using `Object`.

## How was this patch tested?

Added `UTF8StringBufferSuite`

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

Closes #20871 from kiszk/SPARK-23762.
2018-04-12 22:21:30 +08:00
Herman van Hovell c604d659e1 [SPARK-23951][SQL] Use actual java class instead of string representation.
## What changes were proposed in this pull request?
This PR slightly refactors the newly added `ExprValue` API by quite a bit. The following changes are introduced:

1. `ExprValue` now uses the actual class instead of the class name as its type. This should give some more flexibility with generating code in the future.
2. Renamed `StatementValue` to `SimpleExprValue`. The statement concept is broader then an expression (untyped and it cannot be on the right hand side of an assignment), and this was not really what we were using it for. I have added a top level `JavaCode` trait that can be used in the future to reinstate (no pun intended) a statement a-like code fragment.
3. Added factory methods to the `JavaCode` companion object to make it slightly less verbose to create `JavaCode`/`ExprValue` objects. This is also what makes the diff quite large.
4. Added one more factory method to `ExprCode` to make it easier to create code-less expressions.

## How was this patch tested?
Existing tests.

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

Closes #21026 from hvanhovell/SPARK-23951.
2018-04-11 20:11:03 +08:00
Gengliang Wang e179658914 [SPARK-19724][SQL][FOLLOW-UP] Check location of managed table when ignoreIfExists is true
## What changes were proposed in this pull request?

In the PR #20886, I mistakenly check the table location only when `ignoreIfExists` is false, which was following the original deprecated PR.
That was wrong. When `ignoreIfExists` is true and the target table doesn't exist, we should also check the table location. In other word, **`ignoreIfExists` has nothing to do with table location validation**.
This is a follow-up PR to fix the mistake.

## How was this patch tested?

Add one unit test.

Author: Gengliang Wang <gengliang.wang@databricks.com>

Closes #21001 from gengliangwang/SPARK-19724-followup.
2018-04-10 09:33:09 -07:00
Herman van Hovell 3323b156f9 [SPARK-23864][SQL] Add unsafe object writing to UnsafeWriter
## What changes were proposed in this pull request?
This PR moves writing of `UnsafeRow`, `UnsafeArrayData` & `UnsafeMapData` out of the `GenerateUnsafeProjection`/`InterpretedUnsafeProjection` classes into the `UnsafeWriter` interface. This cleans up the code a little bit, and it should also result in less byte code for the code generated path.

## How was this patch tested?
Existing tests

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

Closes #20986 from hvanhovell/SPARK-23864.
2018-04-10 17:32:00 +02:00
Herman van Hovell 6498884154 [SPARK-23898][SQL] Simplify add & subtract code generation
## What changes were proposed in this pull request?
Code generation for the `Add` and `Subtract` expressions was not done using the `BinaryArithmetic.doCodeGen` method because these expressions also support `CalendarInterval`. This leads to a bit of duplication.

This PR gets rid of that duplication by adding `calendarIntervalMethod` to `BinaryArithmetic` and doing the code generation for `CalendarInterval` in `BinaryArithmetic` instead.

## How was this patch tested?
Existing tests.

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

Closes #21005 from hvanhovell/SPARK-23898.
2018-04-09 21:49:49 -07:00
Kris Mok f94f3624ea [SPARK-23947][SQL] Add hashUTF8String convenience method to hasher classes
## What changes were proposed in this pull request?

Add `hashUTF8String()` to the hasher classes to allow Spark SQL codegen to generate cleaner code for hashing `UTF8String`s. No change in behavior otherwise.

Although with the introduction of SPARK-10399, the code size for hashing `UTF8String` is already smaller, it's still good to extract a separate function in the hasher classes so that the generated code can stay clean.

## How was this patch tested?

Existing tests.

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

Closes #21016 from rednaxelafx/hashutf8.
2018-04-09 21:07:28 -07:00
Liang-Chi Hsieh 7c1654e215 [SPARK-22856][SQL] Add wrappers for codegen output and nullability
## What changes were proposed in this pull request?

The codegen output of `Expression`, aka `ExprCode`, now encapsulates only strings of output value (`value`) and nullability (`isNull`). It makes difficulty for us to know what the output really is. I think it is better if we can add wrappers for the value and nullability that let us to easily know that.

## How was this patch tested?

Existing tests.

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

Closes #20043 from viirya/SPARK-22856.
2018-04-09 11:54:35 -07:00
Kazuaki Ishizaki 8d40a79a07 [SPARK-23893][CORE][SQL] Avoid possible integer overflow in multiplication
## What changes were proposed in this pull request?

This PR avoids possible overflow at an operation `long = (long)(int * int)`. The multiplication of large positive integer values may set one to MSB. This leads to a negative value in long while we expected a positive value (e.g. `0111_0000_0000_0000 * 0000_0000_0000_0010`).

This PR performs long cast before the multiplication to avoid this situation.

## How was this patch tested?

Existing UTs

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

Closes #21002 from kiszk/SPARK-23893.
2018-04-08 20:40:27 +02:00
Kazuaki Ishizaki b6935ffb4d [SPARK-10399][SPARK-23879][HOTFIX] Fix Java lint errors
## What changes were proposed in this pull request?

This PR fixes the following errors in [Java lint](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Compile/job/spark-master-lint/7717/console) after #19222 has been merged. These errors were pointed by ueshin .

```
[ERROR] src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java:[57] (sizes) LineLength: Line is longer than 100 characters (found 106).
[ERROR] src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java:[26,8] (imports) UnusedImports: Unused import - org.apache.spark.unsafe.Platform.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java:[23,10] (modifier) ModifierOrder: 'public' modifier out of order with the JLS suggestions.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[64,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[69,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[74,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[79,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[84,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[89,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[94,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[99,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[104,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[109,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[114,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[119,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[124,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[129,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[60,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[65,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[70,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[75,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[80,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[85,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[90,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[95,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[100,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[105,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[110,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[115,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[120,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[125,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java:[114,16] (modifier) ModifierOrder: 'static' modifier out of order with the JLS suggestions.
[ERROR] src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java:[20,8] (imports) UnusedImports: Unused import - org.apache.spark.unsafe.Platform.
[ERROR] src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java:[30,8] (imports) UnusedImports: Unused import - org.apache.spark.unsafe.memory.MemoryBlock.
[ERROR] src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java:[126,15] (naming) MethodName: Method name 'ByteArrayMemoryBlockTest' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
[ERROR] src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java:[143,15] (naming) MethodName: Method name 'OnHeapMemoryBlockTest' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
[ERROR] src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java:[160,15] (naming) MethodName: Method name 'OffHeapArrayMemoryBlockTest' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
[ERROR] src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java:[19,8] (imports) UnusedImports: Unused import - com.google.common.primitives.Ints.
[ERROR] src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java:[21,8] (imports) UnusedImports: Unused import - org.apache.spark.unsafe.Platform.
[ERROR] src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java:[20,8] (imports) UnusedImports: Unused import - org.apache.spark.unsafe.Platform.
```

## How was this patch tested?

Existing UTs

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

Closes #20991 from kiszk/SPARK-10399-jlint.
2018-04-06 10:23:26 -07:00
Gengliang Wang 249007e37f [SPARK-19724][SQL] create a managed table with an existed default table should throw an exception
## What changes were proposed in this pull request?
This PR is to finish https://github.com/apache/spark/pull/17272

This JIRA is a follow up work after SPARK-19583

As we discussed in that PR

The following DDL for a managed table with an existed default location should throw an exception:

CREATE TABLE ... (PARTITIONED BY ...) AS SELECT ...
CREATE TABLE ... (PARTITIONED BY ...)
Currently there are some situations which are not consist with above logic:

CREATE TABLE ... (PARTITIONED BY ...) succeed with an existed default location
situation: for both hive/datasource(with HiveExternalCatalog/InMemoryCatalog)

CREATE TABLE ... (PARTITIONED BY ...) AS SELECT ...
situation: hive table succeed with an existed default location

This PR is going to make above two situations consist with the logic that it should throw an exception
with an existed default location.
## How was this patch tested?

unit test added

Author: Gengliang Wang <gengliang.wang@databricks.com>

Closes #20886 from gengliangwang/pr-17272.
2018-04-05 20:19:25 -07:00
JiahuiJiang d65e531b44 [SPARK-23823][SQL] Keep origin in transformExpression
Fixes https://issues.apache.org/jira/browse/SPARK-23823

Keep origin for all the methods using transformExpression

## What changes were proposed in this pull request?

Keep origin in transformExpression

## How was this patch tested?

Manually tested that this fixes https://issues.apache.org/jira/browse/SPARK-23823 and columns have correct origins after Analyzer.analyze

Author: JiahuiJiang <jjiang@palantir.com>
Author: Jiahui Jiang <jjiang@palantir.com>

Closes #20961 from JiahuiJiang/jj/keep-origin.
2018-04-05 20:06:08 -07:00
Kazuaki Ishizaki 4807d381bb [SPARK-10399][CORE][SQL] Introduce multiple MemoryBlocks to choose several types of memory block
## What changes were proposed in this pull request?

This PR allows us to use one of several types of `MemoryBlock`, such as byte array, int array, long array, or `java.nio.DirectByteBuffer`. To use `java.nio.DirectByteBuffer` allows to have off heap memory which is automatically deallocated by JVM. `MemoryBlock`  class has primitive accessors like `Platform.getInt()`, `Platform.putint()`, or `Platform.copyMemory()`.

This PR uses `MemoryBlock` for `OffHeapColumnVector`, `UTF8String`, and other places. This PR can improve performance of operations involving memory accesses (e.g. `UTF8String.trim`) by 1.8x.

For now, this PR does not use `MemoryBlock` for `BufferHolder` based on cloud-fan's [suggestion](https://github.com/apache/spark/pull/11494#issuecomment-309694290).

Since this PR is a successor of #11494, close #11494. Many codes were ported from #11494. Many efforts were put here. **I think this PR should credit to yzotov.**

This PR can achieve **1.1-1.4x performance improvements** for  operations in `UTF8String` or `Murmur3_x86_32`. Other operations are almost comparable performances.

Without this PR
```
OpenJDK 64-Bit Server VM 1.8.0_121-8u121-b13-0ubuntu1.16.04.2-b13 on Linux 4.4.0-22-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
OpenJDK 64-Bit Server VM 1.8.0_121-8u121-b13-0ubuntu1.16.04.2-b13 on Linux 4.4.0-22-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
Hash byte arrays with length 268435487:  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Murmur3_x86_32                                 526 /  536          0.0   131399881.5       1.0X

UTF8String benchmark:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
hashCode                                       525 /  552       1022.6           1.0       1.0X
substring                                      414 /  423       1298.0           0.8       1.3X
```

With this PR
```
OpenJDK 64-Bit Server VM 1.8.0_121-8u121-b13-0ubuntu1.16.04.2-b13 on Linux 4.4.0-22-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
Hash byte arrays with length 268435487:  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Murmur3_x86_32                                 474 /  488          0.0   118552232.0       1.0X

UTF8String benchmark:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
hashCode                                       476 /  480       1127.3           0.9       1.0X
substring                                      287 /  291       1869.9           0.5       1.7X
```

Benchmark program
```
test("benchmark Murmur3_x86_32") {
  val length = 8192 * 32768 + 31
  val seed = 42L
  val iters = 1 << 2
  val random = new Random(seed)
  val arrays = Array.fill[MemoryBlock](numArrays) {
    val bytes = new Array[Byte](length)
    random.nextBytes(bytes)
    new ByteArrayMemoryBlock(bytes, Platform.BYTE_ARRAY_OFFSET, length)
  }

  val benchmark = new Benchmark("Hash byte arrays with length " + length,
    iters * numArrays, minNumIters = 20)
  benchmark.addCase("HiveHasher") { _: Int =>
    var sum = 0L
    for (_ <- 0L until iters) {
      sum += HiveHasher.hashUnsafeBytesBlock(
        arrays(i), Platform.BYTE_ARRAY_OFFSET, length)
    }
  }
  benchmark.run()
}

test("benchmark UTF8String") {
  val N = 512 * 1024 * 1024
  val iters = 2
  val benchmark = new Benchmark("UTF8String benchmark", N, minNumIters = 20)
  val str0 = new java.io.StringWriter() { { for (i <- 0 until N) { write(" ") } } }.toString
  val s0 = UTF8String.fromString(str0)
  benchmark.addCase("hashCode") { _: Int =>
    var h: Int = 0
    for (_ <- 0L until iters) { h += s0.hashCode }
  }
  benchmark.addCase("substring") { _: Int =>
    var s: UTF8String = null
    for (_ <- 0L until iters) { s = s0.substring(N / 2 - 5, N / 2 + 5) }
  }
  benchmark.run()
}
```

I run [this benchmark program](https://gist.github.com/kiszk/94f75b506c93a663bbbc372ffe8f05de) using [the commit](ee5a79861c). I got the following results:

```
OpenJDK 64-Bit Server VM 1.8.0_151-8u151-b12-0ubuntu0.16.04.2-b12 on Linux 4.4.0-66-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
Memory access benchmarks:                Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
ByteArrayMemoryBlock get/putInt()              220 /  221        609.3           1.6       1.0X
Platform get/putInt(byte[])                    220 /  236        610.9           1.6       1.0X
Platform get/putInt(Object)                    492 /  494        272.8           3.7       0.4X
OnHeapMemoryBlock get/putLong()                322 /  323        416.5           2.4       0.7X
long[]                                         221 /  221        608.0           1.6       1.0X
Platform get/putLong(long[])                   321 /  321        418.7           2.4       0.7X
Platform get/putLong(Object)                   561 /  563        239.2           4.2       0.4X
```

I also run [this benchmark program](https://gist.github.com/kiszk/5fdb4e03733a5d110421177e289d1fb5) for comparing performance of `Platform.copyMemory()`.
```
OpenJDK 64-Bit Server VM 1.8.0_151-8u151-b12-0ubuntu0.16.04.2-b12 on Linux 4.4.0-66-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
Platform copyMemory:                     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Object to Object                              1961 / 1967          8.6         116.9       1.0X
System.arraycopy Object to Object             1917 / 1921          8.8         114.3       1.0X
byte array to byte array                      1961 / 1968          8.6         116.9       1.0X
System.arraycopy byte array to byte array      1909 / 1937          8.8         113.8       1.0X
int array to int array                        1921 / 1990          8.7         114.5       1.0X
double array to double array                  1918 / 1923          8.7         114.3       1.0X
Object to byte array                          1961 / 1967          8.6         116.9       1.0X
Object to short array                         1965 / 1972          8.5         117.1       1.0X
Object to int array                           1910 / 1915          8.8         113.9       1.0X
Object to float array                         1971 / 1978          8.5         117.5       1.0X
Object to double array                        1919 / 1944          8.7         114.4       1.0X
byte array to Object                          1959 / 1967          8.6         116.8       1.0X
int array to Object                           1961 / 1970          8.6         116.9       1.0X
double array to Object                        1917 / 1924          8.8         114.3       1.0X
```

These results show three facts:
1. According to the second/third or sixth/seventh results in the first experiment, if we use `Platform.get/putInt(Object)`, we achieve more than 2x worse performance than `Platform.get/putInt(byte[])` with concrete type (i.e. `byte[]`).
2. According to the second/third or fourth/fifth/sixth results in the first experiment, the fastest way to access an array element on Java heap is `array[]`. **Cons of `array[]` is that it is not possible to support unaligned-8byte access.**
3. According to the first/second/third or fourth/sixth/seventh results in the first experiment, `getInt()/putInt() or getLong()/putLong()` in subclasses of `MemoryBlock` can achieve comparable performance to `Platform.get/putInt()` or `Platform.get/putLong()` with concrete type (second or sixth result). There is no overhead regarding virtual call.
4. According to results in the second experiment, for `Platform.copy()`, to pass `Object` can achieve the same performance as to pass any type of primitive array as source or destination.
5. According to second/fourth results in the second experiment, `Platform.copy()` can achieve the same performance as `System.arrayCopy`. **It would be good to use `Platform.copy()` since `Platform.copy()` can take any types for src and dst.**

We are incrementally replace `Platform.get/putXXX` with `MemoryBlock.get/putXXX`. This is because we have two advantages.
1) Achieve better performance due to having a concrete type for an array.
2) Use simple OO design instead of passing `Object`
It is easy to use `MemoryBlock` in `InternalRow`, `BufferHolder`, `TaskMemoryManager`, and others that are already abstracted. It is not easy to use `MemoryBlock` in utility classes related to hashing or others.

Other candidates are
- UnsafeRow, UnsafeArrayData, UnsafeMapData, SpecificUnsafeRowJoiner
- UTF8StringBuffer
- BufferHolder
- TaskMemoryManager
- OnHeapColumnVector
- BytesToBytesMap
- CachedBatch
- classes for hash
- others.

## How was this patch tested?

Added `UnsafeMemoryAllocator`

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

Closes #19222 from kiszk/SPARK-10399.
2018-04-06 10:13:59 +08:00
Liang-Chi Hsieh d9ca1c906b [SPARK-23593][SQL] Add interpreted execution for InitializeJavaBean expression
## What changes were proposed in this pull request?

Add interpreted execution for `InitializeJavaBean` expression.

## How was this patch tested?

Added unit test.

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

Closes #20985 from viirya/SPARK-23593-2.
2018-04-05 20:43:05 +02:00
Herman van Hovell b2329fb1fc Revert "[SPARK-23593][SQL] Add interpreted execution for InitializeJavaBean expression"
This reverts commit c5c8b54404.
2018-04-05 13:57:41 +02:00
Kazuaki Ishizaki 1822ecda51 [SPARK-23582][SQL] StaticInvoke should support interpreted execution
## What changes were proposed in this pull request?

This pr added interpreted execution for `StaticInvoke`.

## How was this patch tested?

Added tests in `ObjectExpressionsSuite`.

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

Closes #20753 from kiszk/SPARK-23582.
2018-04-05 13:47:06 +02:00
Liang-Chi Hsieh c5c8b54404 [SPARK-23593][SQL] Add interpreted execution for InitializeJavaBean expression
## What changes were proposed in this pull request?

Add interpreted execution for `InitializeJavaBean` expression.

## How was this patch tested?

Added unit test.

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

Closes #20756 from viirya/SPARK-23593.
2018-04-05 13:39:45 +02:00
Kazuaki Ishizaki a35523653c [SPARK-23583][SQL] Invoke should support interpreted execution
## What changes were proposed in this pull request?

This pr added interpreted execution for `Invoke`.

## How was this patch tested?

Added tests in `ObjectExpressionsSuite`.

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

Closes #20797 from kiszk/SPARK-28583.
2018-04-04 18:36:15 +02:00
Takeshi Yamamuro 5197562afe [SPARK-21351][SQL] Update nullability based on children's output
## What changes were proposed in this pull request?
This pr added a new optimizer rule `UpdateNullabilityInAttributeReferences ` to update the nullability that `Filter` changes when having `IsNotNull`. In the master, optimized plans do not respect the nullability when `Filter` has `IsNotNull`. This wrongly generates unnecessary code. For example:

```
scala> val df = Seq((Some(1), Some(2))).toDF("a", "b")
scala> val bIsNotNull = df.where($"b" =!= 2).select($"b")
scala> val targetQuery = bIsNotNull.distinct
scala> val targetQuery.queryExecution.optimizedPlan.output(0).nullable
res5: Boolean = true

scala> targetQuery.debugCodegen
Found 2 WholeStageCodegen subtrees.
== Subtree 1 / 2 ==
*HashAggregate(keys=[b#19], functions=[], output=[b#19])
+- Exchange hashpartitioning(b#19, 200)
   +- *HashAggregate(keys=[b#19], functions=[], output=[b#19])
      +- *Project [_2#16 AS b#19]
         +- *Filter isnotnull(_2#16)
            +- LocalTableScan [_1#15, _2#16]

Generated code:
...
/* 124 */   protected void processNext() throws java.io.IOException {
...
/* 132 */     // output the result
/* 133 */
/* 134 */     while (agg_mapIter.next()) {
/* 135 */       wholestagecodegen_numOutputRows.add(1);
/* 136 */       UnsafeRow agg_aggKey = (UnsafeRow) agg_mapIter.getKey();
/* 137 */       UnsafeRow agg_aggBuffer = (UnsafeRow) agg_mapIter.getValue();
/* 138 */
/* 139 */       boolean agg_isNull4 = agg_aggKey.isNullAt(0);
/* 140 */       int agg_value4 = agg_isNull4 ? -1 : (agg_aggKey.getInt(0));
/* 141 */       agg_rowWriter1.zeroOutNullBytes();
/* 142 */
                // We don't need this NULL check because NULL is filtered out in `$"b" =!=2`
/* 143 */       if (agg_isNull4) {
/* 144 */         agg_rowWriter1.setNullAt(0);
/* 145 */       } else {
/* 146 */         agg_rowWriter1.write(0, agg_value4);
/* 147 */       }
/* 148 */       append(agg_result1);
/* 149 */
/* 150 */       if (shouldStop()) return;
/* 151 */     }
/* 152 */
/* 153 */     agg_mapIter.close();
/* 154 */     if (agg_sorter == null) {
/* 155 */       agg_hashMap.free();
/* 156 */     }
/* 157 */   }
/* 158 */
/* 159 */ }
```

In the line 143, we don't need this NULL check because NULL is filtered out in `$"b" =!=2`.
This pr could remove this NULL check;

```
scala> val targetQuery.queryExecution.optimizedPlan.output(0).nullable
res5: Boolean = false

scala> targetQuery.debugCodegen
...
Generated code:
...
/* 144 */   protected void processNext() throws java.io.IOException {
...
/* 152 */     // output the result
/* 153 */
/* 154 */     while (agg_mapIter.next()) {
/* 155 */       wholestagecodegen_numOutputRows.add(1);
/* 156 */       UnsafeRow agg_aggKey = (UnsafeRow) agg_mapIter.getKey();
/* 157 */       UnsafeRow agg_aggBuffer = (UnsafeRow) agg_mapIter.getValue();
/* 158 */
/* 159 */       int agg_value4 = agg_aggKey.getInt(0);
/* 160 */       agg_rowWriter1.write(0, agg_value4);
/* 161 */       append(agg_result1);
/* 162 */
/* 163 */       if (shouldStop()) return;
/* 164 */     }
/* 165 */
/* 166 */     agg_mapIter.close();
/* 167 */     if (agg_sorter == null) {
/* 168 */       agg_hashMap.free();
/* 169 */     }
/* 170 */   }
```

## How was this patch tested?
Added `UpdateNullabilityInAttributeReferencesSuite` for unit tests.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #18576 from maropu/SPARK-21351.
2018-04-04 14:39:19 +08:00
Robert Kruszewski 5cfd5fabcd [SPARK-23802][SQL] PropagateEmptyRelation can leave query plan in unresolved state
## What changes were proposed in this pull request?

Add cast to nulls introduced by PropagateEmptyRelation so in cases they're part of coalesce they will not break its type checking rules

## How was this patch tested?

Added unit test

Author: Robert Kruszewski <robertk@palantir.com>

Closes #20914 from robert3005/rk/propagate-empty-fix.
2018-04-03 17:25:54 -07:00
Liang-Chi Hsieh 1035aaa617 [SPARK-23587][SQL] Add interpreted execution for MapObjects expression
## What changes were proposed in this pull request?

Add interpreted execution for `MapObjects` expression.

## How was this patch tested?

Added unit test.

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

Closes #20771 from viirya/SPARK-23587.
2018-04-04 01:36:58 +02:00
Kazuaki Ishizaki a7c19d9c21 [SPARK-23713][SQL] Cleanup UnsafeWriter and BufferHolder classes
## What changes were proposed in this pull request?

This PR implemented the following cleanups related to  `UnsafeWriter` class:
- Remove code duplication between `UnsafeRowWriter` and `UnsafeArrayWriter`
- Make `BufferHolder` class internal by delegating its accessor methods to `UnsafeWriter`
- Replace `UnsafeRow.setTotalSize(...)` with `UnsafeRowWriter.setTotalSize()`

## How was this patch tested?

Tested by existing UTs

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

Closes #20850 from kiszk/SPARK-23713.
2018-04-02 21:48:44 +02:00
gatorsmile bc8d093117 [SPARK-23500][SQL][FOLLOWUP] Fix complex type simplification rules to apply to entire plan
## What changes were proposed in this pull request?
This PR is to improve the test coverage of the original PR https://github.com/apache/spark/pull/20687

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20911 from gatorsmile/addTests.
2018-03-30 23:21:07 +08:00
yucai b02e76cbff [SPARK-23727][SQL] Support for pushing down filters for DateType in parquet
## What changes were proposed in this pull request?

This PR supports for pushing down filters for DateType in parquet

## How was this patch tested?

Added UT and tested in local.

Author: yucai <yyu1@ebay.com>

Closes #20851 from yucai/SPARK-23727.
2018-03-30 15:07:38 +08:00
hyukjinkwon 34c4b9c57e [SPARK-23765][SQL] Supports custom line separator for json datasource
## What changes were proposed in this pull request?

This PR proposes to add lineSep option for a configurable line separator in text datasource.
It supports this option by using `LineRecordReader`'s functionality with passing it to the constructor.

The approach is similar with https://github.com/apache/spark/pull/20727; however, one main difference is, it uses text datasource's `lineSep` option to parse line by line in JSON's schema inference.

## How was this patch tested?

Manually tested and unit tests were added.

Author: hyukjinkwon <gurwls223@apache.org>
Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20877 from HyukjinKwon/linesep-json.
2018-03-28 19:49:27 +08:00
Liang-Chi Hsieh 35997b59f3 [SPARK-23794][SQL] Make UUID as stateful expression
## What changes were proposed in this pull request?

The UUID() expression is stateful and should implement the `Stateful` trait instead of the `Nondeterministic` trait.

## How was this patch tested?

Added test.

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

Closes #20912 from viirya/SPARK-23794.
2018-03-27 14:49:50 +02:00
Kazuaki Ishizaki e4bec7cb88 [SPARK-23549][SQL] Cast to timestamp when comparing timestamp with date
## What changes were proposed in this pull request?

This PR fixes an incorrect comparison in SQL between timestamp and date. This is because both of them are casted to `string` and then are compared lexicographically. This implementation shows `false` regarding this query `spark.sql("select cast('2017-03-01 00:00:00' as timestamp) between cast('2017-02-28' as date) and cast('2017-03-01' as date)").show`.

This PR shows `true` for this query by casting `date("2017-03-01")` to `timestamp("2017-03-01 00:00:00")`.

(Please fill in changes proposed in this fix)

## How was this patch tested?

Added new UTs to `TypeCoercionSuite`.

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

Closes #20774 from kiszk/SPARK-23549.
2018-03-25 16:38:49 -07:00
Liang-Chi Hsieh 4d37008c78 [SPARK-23599][SQL] Use RandomUUIDGenerator in Uuid expression
## What changes were proposed in this pull request?

As stated in Jira, there are problems with current `Uuid` expression which uses `java.util.UUID.randomUUID` for UUID generation.

This patch uses the newly added `RandomUUIDGenerator` for UUID generation. So we can make `Uuid` deterministic between retries.

## How was this patch tested?

Added unit tests.

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

Closes #20861 from viirya/SPARK-23599-2.
2018-03-22 19:57:32 +01:00
Kris Mok 95e51ff849 [SPARK-23760][SQL] CodegenContext.withSubExprEliminationExprs should save/restore CSE state correctly
## What changes were proposed in this pull request?

Fixed `CodegenContext.withSubExprEliminationExprs()` so that it saves/restores CSE state correctly.

## How was this patch tested?

Added new unit test to verify that the old CSE state is indeed saved and restored around the `withSubExprEliminationExprs()` call. Manually verified that this test fails without this patch.

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

Closes #20870 from rednaxelafx/codegen-subexpr-fix.
2018-03-21 21:21:36 -07:00
Takeshi Yamamuro 983e8d9d64 [SPARK-23666][SQL] Do not display exprIds of Alias in user-facing info.
## What changes were proposed in this pull request?
To drop `exprId`s for `Alias` in user-facing info., this pr added an entry for `Alias` in `NonSQLExpression.sql`

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20827 from maropu/SPARK-23666.
2018-03-20 23:17:49 -07:00
Henry Robinson 477d6bd726 [SPARK-23500][SQL] Fix complex type simplification rules to apply to entire plan
## What changes were proposed in this pull request?

Complex type simplification optimizer rules were not applied to the
entire plan, just the expressions reachable from the root node. This
patch fixes the rules to transform the entire plan.

## How was this patch tested?

New unit test + ran sql / core tests.

Author: Henry Robinson <henry@apache.org>
Author: Henry Robinson <henry@cloudera.com>

Closes #20687 from henryr/spark-25000.
2018-03-20 13:27:50 -07:00
Liang-Chi Hsieh 4de638c197 [SPARK-23599][SQL] Add a UUID generator from Pseudo-Random Numbers
## What changes were proposed in this pull request?

This patch adds a UUID generator from Pseudo-Random Numbers. We can use it later to have deterministic `UUID()` expression.

## How was this patch tested?

Added unit tests.

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

Closes #20817 from viirya/SPARK-23599.
2018-03-19 09:41:43 +01:00
Herman van Hovell 88d8de9260 [SPARK-23581][SQL] Add interpreted unsafe projection
## What changes were proposed in this pull request?
We currently can only create unsafe rows using code generation. This is a problem for situations in which code generation fails. There is no fallback, and as a result we cannot execute the query.

This PR adds an interpreted version of `UnsafeProjection`. The implementation is modeled after `InterpretedMutableProjection`. It stores the expression results in a `GenericInternalRow`, and it then uses a conversion function to convert the `GenericInternalRow` into an `UnsafeRow`.

This PR does not implement the actual code generated to interpreted fallback logic. This will be done in a follow-up.

## How was this patch tested?
I am piggybacking on exiting `UnsafeProjection` tests, and I have added an interpreted version for each of these.

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

Closes #20750 from hvanhovell/SPARK-23581.
2018-03-16 18:28:16 +01:00
Kazuaki Ishizaki 23370554d0 [SPARK-23656][TEST] Perform assertions in XXH64Suite.testKnownByteArrayInputs() on big endian platform, too
## What changes were proposed in this pull request?

This PR enables assertions in `XXH64Suite.testKnownByteArrayInputs()` on big endian platform, too. The current implementation performs them only on little endian platform. This PR increase test coverage of big endian platform.

## How was this patch tested?

Updated `XXH64Suite`
Tested on big endian platform using JIT compiler or interpreter `-Xint`.

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

Closes #20804 from kiszk/SPARK-23656.
2018-03-13 15:20:09 +01:00
Xiayun Sun b304e07e06 [SPARK-23462][SQL] improve missing field error message in StructType
## What changes were proposed in this pull request?

The error message ```s"""Field "$name" does not exist."""``` is thrown when looking up an unknown field in StructType. In the error message, we should also contain the information about which columns/fields exist in this struct.

## How was this patch tested?

Added new unit tests.

Note: I created a new `StructTypeSuite.scala` as I couldn't find an existing suite that's suitable to place these tests. I may be missing something so feel free to propose new locations.

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

Author: Xiayun Sun <xiayunsun@gmail.com>

Closes #20649 from xysun/SPARK-23462.
2018-03-12 22:13:28 +09:00
Michał Świtakowski 2ca9bb083c [SPARK-23173][SQL] Avoid creating corrupt parquet files when loading data from JSON
## What changes were proposed in this pull request?

The from_json() function accepts an additional parameter, where the user might specify the schema. The issue is that the specified schema might not be compatible with data. In particular, the JSON data might be missing data for fields declared as non-nullable in the schema. The from_json() function does not verify the data against such errors. When data with missing fields is sent to the parquet encoder, there is no verification either. The end results is a corrupt parquet file.

To avoid corruptions, make sure that all fields in the user-specified schema are set to be nullable.
Since this changes the behavior of a public function, we need to include it in release notes.
The behavior can be reverted by setting `spark.sql.fromJsonForceNullableSchema=false`

## How was this patch tested?

Added two new tests.

Author: Michał Świtakowski <michal.switakowski@databricks.com>

Closes #20694 from mswit-databricks/SPARK-23173.
2018-03-09 14:29:31 -08:00
Marco Gaido e7bbca8896 [SPARK-23602][SQL] PrintToStderr prints value also in interpreted mode
## What changes were proposed in this pull request?

`PrintToStderr` was doing what is it supposed to only when code generation is enabled.
The PR adds the same behavior in interpreted mode too.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20773 from mgaido91/SPARK-23602.
2018-03-08 22:02:28 +01:00
Marco Gaido ea480990e7 [SPARK-23628][SQL] calculateParamLength should not return 1 + num of epressions
## What changes were proposed in this pull request?

There was a bug in `calculateParamLength` which caused it to return always 1 + the number of expressions. This could lead to Exceptions especially with expressions of type long.

## How was this patch tested?

added UT + fixed previous UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20772 from mgaido91/SPARK-23628.
2018-03-08 11:09:15 -08:00
Marco Gaido 92e7ecbbbd [SPARK-23592][SQL] Add interpreted execution to DecodeUsingSerializer
## What changes were proposed in this pull request?

The PR adds interpreted execution to DecodeUsingSerializer.

## How was this patch tested?
added UT

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

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20760 from mgaido91/SPARK-23592.
2018-03-08 14:18:14 +01:00
hyukjinkwon d6632d185e [SPARK-23380][PYTHON] Adds a conf for Arrow fallback in toPandas/createDataFrame with Pandas DataFrame
## What changes were proposed in this pull request?

This PR adds a configuration to control the fallback of Arrow optimization for `toPandas` and `createDataFrame` with Pandas DataFrame.

## How was this patch tested?

Manually tested and unit tests added.

You can test this by:

**`createDataFrame`**

```python
spark.conf.set("spark.sql.execution.arrow.enabled", False)
pdf = spark.createDataFrame([[{'a': 1}]]).toPandas()
spark.conf.set("spark.sql.execution.arrow.enabled", True)
spark.conf.set("spark.sql.execution.arrow.fallback.enabled", True)
spark.createDataFrame(pdf, "a: map<string, int>")
```

```python
spark.conf.set("spark.sql.execution.arrow.enabled", False)
pdf = spark.createDataFrame([[{'a': 1}]]).toPandas()
spark.conf.set("spark.sql.execution.arrow.enabled", True)
spark.conf.set("spark.sql.execution.arrow.fallback.enabled", False)
spark.createDataFrame(pdf, "a: map<string, int>")
```

**`toPandas`**

```python
spark.conf.set("spark.sql.execution.arrow.enabled", True)
spark.conf.set("spark.sql.execution.arrow.fallback.enabled", True)
spark.createDataFrame([[{'a': 1}]]).toPandas()
```

```python
spark.conf.set("spark.sql.execution.arrow.enabled", True)
spark.conf.set("spark.sql.execution.arrow.fallback.enabled", False)
spark.createDataFrame([[{'a': 1}]]).toPandas()
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20678 from HyukjinKwon/SPARK-23380-conf.
2018-03-08 20:22:07 +09:00
Marco Gaido aff7d81cb7 [SPARK-23591][SQL] Add interpreted execution to EncodeUsingSerializer
## What changes were proposed in this pull request?

The PR adds interpreted execution to EncodeUsingSerializer.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20751 from mgaido91/SPARK-23591.
2018-03-07 18:31:59 +01:00
Takeshi Yamamuro 33c2cb22b3 [SPARK-23611][SQL] Add a helper function to check exception for expr evaluation
## What changes were proposed in this pull request?
This pr added a helper function in `ExpressionEvalHelper` to check exceptions in all the path of expression evaluation.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20748 from maropu/SPARK-23611.
2018-03-07 13:10:51 +01:00
Marco Gaido 4c587eb488 [SPARK-23590][SQL] Add interpreted execution to CreateExternalRow
## What changes were proposed in this pull request?

The PR adds interpreted execution to CreateExternalRow

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20749 from mgaido91/SPARK-23590.
2018-03-06 17:42:17 +01:00
Takeshi Yamamuro e8a259d66d [SPARK-23594][SQL] GetExternalRowField should support interpreted execution
## What changes were proposed in this pull request?
This pr added interpreted execution for `GetExternalRowField`.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20746 from maropu/SPARK-23594.
2018-03-06 13:55:13 +01:00
Marco Gaido f6b49f9d1b [SPARK-23586][SQL] Add interpreted execution to WrapOption
## What changes were proposed in this pull request?

The PR adds interpreted execution to WrapOption.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20741 from mgaido91/SPARK-23586_2.
2018-03-06 01:37:51 +01:00
Marco Gaido ba622f45ca [SPARK-23585][SQL] Add interpreted execution to UnwrapOption
## What changes were proposed in this pull request?

The PR adds interpreted execution to UnwrapOption.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20736 from mgaido91/SPARK-23586.
2018-03-05 20:43:03 +01:00
Mihaly Toth a366b950b9 [SPARK-23329][SQL] Fix documentation of trigonometric functions
## What changes were proposed in this pull request?

Provide more details in trigonometric function documentations. Referenced `java.lang.Math` for further details in the descriptions.
## How was this patch tested?

Ran full build, checked generated documentation manually

Author: Mihaly Toth <misutoth@gmail.com>

Closes #20618 from misutoth/trigonometric-doc.
2018-03-05 23:46:40 +09:00
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
Eric Liang a89cdf55fa [SQL][MINOR] XPathDouble prettyPrint should say 'double' not 'float'
## What changes were proposed in this pull request?

It looks like this was incorrectly copied from `XPathFloat` in the class above.

## 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: Eric Liang <ekhliang@gmail.com>

Closes #20730 from ericl/fix-typo-xpath.
2018-03-05 07:32:24 +09:00
Feng Liu 3a4d15e5d2 [SPARK-23518][SQL] Avoid metastore access when the users only want to read and write data frames
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/18944 added one patch, which allowed a spark session to be created when the hive metastore server is down. However, it did not allow running any commands with the spark session. This brings troubles to the user who only wants to read / write data frames without metastore setup.

## How was this patch tested?

Added some unit tests to read and write data frames based on the original HiveMetastoreLazyInitializationSuite.

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

Author: Feng Liu <fengliu@databricks.com>

Closes #20681 from liufengdb/completely-lazy.
2018-03-02 10:38:50 -08: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
Liang-Chi Hsieh b14993e1fc [SPARK-23448][SQL] Clarify JSON and CSV parser behavior in document
## What changes were proposed in this pull request?

Clarify JSON and CSV reader behavior in document.

JSON doesn't support partial results for corrupted records.
CSV only supports partial results for the records with more or less tokens.

## How was this patch tested?

Pass existing tests.

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

Closes #20666 from viirya/SPARK-23448-2.
2018-02-28 11:00:54 +09:00
gatorsmile 414ee867ba [SPARK-23523][SQL] Fix the incorrect result caused by the rule OptimizeMetadataOnlyQuery
## What changes were proposed in this pull request?
```Scala
val tablePath = new File(s"${path.getCanonicalPath}/cOl3=c/cOl1=a/cOl5=e")
 Seq(("a", "b", "c", "d", "e")).toDF("cOl1", "cOl2", "cOl3", "cOl4", "cOl5")
 .write.json(tablePath.getCanonicalPath)
 val df = spark.read.json(path.getCanonicalPath).select("CoL1", "CoL5", "CoL3").distinct()
 df.show()
```

It generates a wrong result.
```
[c,e,a]
```

We have a bug in the rule `OptimizeMetadataOnlyQuery `. We should respect the attribute order in the original leaf node. This PR is to fix it.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20684 from gatorsmile/optimizeMetadataOnly.
2018-02-27 08:44:25 -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
Kazuaki Ishizaki 95e25ed1a8 [SPARK-23424][SQL] Add codegenStageId in comment
## What changes were proposed in this pull request?

This PR always adds `codegenStageId` in comment of the generated class. This is a replication of #20419  for post-Spark 2.3.
Closes #20419

```
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage1(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=1
/* 006 */ final class GeneratedIteratorForCodegenStage1 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
...
```

## How was this patch tested?

Existing tests

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

Closes #20612 from kiszk/SPARK-23424.
2018-02-21 11:26:06 +08:00
Dongjoon Hyun 83c008762a [SPARK-23456][SPARK-21783] Turn on native ORC impl and PPD by default
## What changes were proposed in this pull request?

Apache Spark 2.3 introduced `native` ORC supports with vectorization and many fixes. However, it's shipped as a not-default option. This PR enables `native` ORC implementation and predicate-pushdown by default for Apache Spark 2.4. We will improve and stabilize ORC data source before Apache Spark 2.4. And, eventually, Apache Spark will drop old Hive-based ORC code.

## How was this patch tested?

Pass the Jenkins with existing tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20634 from dongjoon-hyun/SPARK-23456.
2018-02-20 09:14:56 -08:00
Kris Mok 15ad4a7f10 [SPARK-23447][SQL] Cleanup codegen template for Literal
## What changes were proposed in this pull request?

Cleaned up the codegen templates for `Literal`s, to make sure that the `ExprCode` returned from `Literal.doGenCode()` has:
1. an empty `code` field;
2. an `isNull` field of either literal `true` or `false`;
3. a `value` field that is just a simple literal/constant.

Before this PR, there are a couple of paths that would return a non-trivial `code` and all of them are actually unnecessary. The `NaN` and `Infinity` constants for `double` and `float` can be accessed through constants directly available so there's no need to add a reference for them.

Also took the opportunity to add a new util method for ease of creating `ExprCode` for inline-able non-null values.

## How was this patch tested?

Existing tests.

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

Closes #20626 from rednaxelafx/codegen-literal.
2018-02-17 10:54:14 +08:00
Tathagata Das 0a73aa31f4 [SPARK-23362][SS] Migrate Kafka Microbatch source to v2
## What changes were proposed in this pull request?
Migrating KafkaSource (with data source v1) to KafkaMicroBatchReader (with data source v2).

Performance comparison:
In a unit test with in-process Kafka broker, I tested the read throughput of V1 and V2 using 20M records in a single partition. They were comparable.

## How was this patch tested?
Existing tests, few modified to be better tests than the existing ones.

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

Closes #20554 from tdas/SPARK-23362.
2018-02-16 14:30:19 -08:00
Dongjoon Hyun 2f0498d1e8 [SPARK-23426][SQL] Use hive ORC impl and disable PPD for Spark 2.3.0
## What changes were proposed in this pull request?

To prevent any regressions, this PR changes ORC implementation to `hive` by default like Spark 2.2.X.
Users can enable `native` ORC. Also, ORC PPD is also restored to `false` like Spark 2.2.X.

![orc_section](https://user-images.githubusercontent.com/9700541/36221575-57a1d702-1173-11e8-89fe-dca5842f4ca7.png)

## How was this patch tested?

Pass all test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20610 from dongjoon-hyun/SPARK-ORC-DISABLE.
2018-02-15 08:55:39 -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
gatorsmile 95e4b49160 [SPARK-23094] Revert [] Fix invalid character handling in JsonDataSource
## What changes were proposed in this pull request?
This PR is to revert the PR https://github.com/apache/spark/pull/20302, because it causes a regression.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20614 from gatorsmile/revertJsonFix.
2018-02-14 23:56:02 -08:00
gatorsmile 2ee76c22b6 [SPARK-23400][SQL] Add a constructors for ScalaUDF
## What changes were proposed in this pull request?

In this upcoming 2.3 release, we changed the interface of `ScalaUDF`. Unfortunately, some Spark packages (e.g., spark-deep-learning) are using our internal class `ScalaUDF`. In the release 2.3, we added new parameters into this class. The users hit the binary compatibility issues and got the exception:

```
> java.lang.NoSuchMethodError: org.apache.spark.sql.catalyst.expressions.ScalaUDF.&lt;init&gt;(Ljava/lang/Object;Lorg/apache/spark/sql/types/DataType;Lscala/collection/Seq;Lscala/collection/Seq;Lscala/Option;)V
```

This PR is to improve the backward compatibility. However, we definitely should not encourage the external packages to use our internal classes. This might make us hard to maintain/develop the codes in Spark.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20591 from gatorsmile/scalaUDF.
2018-02-13 11:56:49 -08:00
Bogdan Raducanu 05d051293f [SPARK-23316][SQL] AnalysisException after max iteration reached for IN query
## What changes were proposed in this pull request?
Added flag ignoreNullability to DataType.equalsStructurally.
The previous semantic is for ignoreNullability=false.
When ignoreNullability=true equalsStructurally ignores nullability of contained types (map key types, value types, array element types, structure field types).
In.checkInputTypes calls equalsStructurally to check if the children types match. They should match regardless of nullability (which is just a hint), so it is now called with ignoreNullability=true.

## How was this patch tested?
New test in SubquerySuite

Author: Bogdan Raducanu <bogdan@databricks.com>

Closes #20548 from bogdanrdc/SPARK-23316.
2018-02-13 09:49:52 -08:00
hyukjinkwon c338c8cf82 [SPARK-23352][PYTHON] Explicitly specify supported types in Pandas UDFs
## What changes were proposed in this pull request?

This PR targets to explicitly specify supported types in Pandas UDFs.
The main change here is to add a deduplicated and explicit type checking in `returnType` ahead with documenting this; however, it happened to fix multiple things.

1. Currently, we don't support `BinaryType` in Pandas UDFs, for example, see:

    ```python
    from pyspark.sql.functions import pandas_udf
    pudf = pandas_udf(lambda x: x, "binary")
    df = spark.createDataFrame([[bytearray(1)]])
    df.select(pudf("_1")).show()
    ```
    ```
    ...
    TypeError: Unsupported type in conversion to Arrow: BinaryType
    ```

    We can document this behaviour for its guide.

2. Also, the grouped aggregate Pandas UDF fails fast on `ArrayType` but seems we can support this case.

    ```python
    from pyspark.sql.functions import pandas_udf, PandasUDFType
    foo = pandas_udf(lambda v: v.mean(), 'array<double>', PandasUDFType.GROUPED_AGG)
    df = spark.range(100).selectExpr("id", "array(id) as value")
    df.groupBy("id").agg(foo("value")).show()
    ```

    ```
    ...
     NotImplementedError: ArrayType, StructType and MapType are not supported with PandasUDFType.GROUPED_AGG
    ```

3. Since we can check the return type ahead, we can fail fast before actual execution.

    ```python
    # we can fail fast at this stage because we know the schema ahead
    pandas_udf(lambda x: x, BinaryType())
    ```

## How was this patch tested?

Manually tested and unit tests for `BinaryType` and `ArrayType(...)` were added.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20531 from HyukjinKwon/pudf-cleanup.
2018-02-12 20:49:36 +09:00
gatorsmile c36fecc3b4 [SPARK-23327][SQL] Update the description and tests of three external API or functions
## What changes were proposed in this pull request?
Update the description and tests of three external API or functions `createFunction `, `length` and `repartitionByRange `

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20495 from gatorsmile/updateFunc.
2018-02-06 16:46:43 -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
Wenchen Fan b9503fcbb3 [SPARK-23312][SQL] add a config to turn off vectorized cache reader
## What changes were proposed in this pull request?

https://issues.apache.org/jira/browse/SPARK-23309 reported a performance regression about cached table in Spark 2.3. While the investigating is still going on, this PR adds a conf to turn off the vectorized cache reader, to unblock the 2.3 release.

## How was this patch tested?

a new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20483 from cloud-fan/cache.
2018-02-02 22:43:28 +08:00
Xingbo Jiang b6b50efc85 [SQL][MINOR] Inline SpecifiedWindowFrame.defaultWindowFrame().
## What changes were proposed in this pull request?

SpecifiedWindowFrame.defaultWindowFrame(hasOrderSpecification, acceptWindowFrame) was designed to handle the cases when some Window functions don't support setting a window frame (e.g. rank). However this param is never used.

We may inline the whole of this function to simplify the code.

## How was this patch tested?

Existing tests.

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

Closes #20463 from jiangxb1987/defaultWindowFrame.
2018-01-31 20:59:19 -08:00
Xingbo Jiang cc41245fa3 [SPARK-23188][SQL] Make vectorized columar reader batch size configurable
## What changes were proposed in this pull request?

This PR include the following changes:
- Make the capacity of `VectorizedParquetRecordReader` configurable;
- Make the capacity of `OrcColumnarBatchReader` configurable;
- Update the error message when required capacity in writable columnar vector cannot be fulfilled.

## How was this patch tested?

N/A

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

Closes #20361 from jiangxb1987/vectorCapacity.
2018-02-01 12:56:07 +08:00
Dilip Biswal 9ff1d96f01 [SPARK-23281][SQL] Query produces results in incorrect order when a composite order by clause refers to both original columns and aliases
## What changes were proposed in this pull request?
Here is the test snippet.
``` SQL
scala> Seq[(Integer, Integer)](
     |         (1, 1),
     |         (1, 3),
     |         (2, 3),
     |         (3, 3),
     |         (4, null),
     |         (5, null)
     |       ).toDF("key", "value").createOrReplaceTempView("src")

scala> sql(
     |         """
     |           |SELECT MAX(value) as value, key as col2
     |           |FROM src
     |           |GROUP BY key
     |           |ORDER BY value desc, key
     |         """.stripMargin).show
+-----+----+
|value|col2|
+-----+----+
|    3|   3|
|    3|   2|
|    3|   1|
| null|   5|
| null|   4|
+-----+----+
```SQL
Here is the explain output :

```SQL
== Parsed Logical Plan ==
'Sort ['value DESC NULLS LAST, 'key ASC NULLS FIRST], true
+- 'Aggregate ['key], ['MAX('value) AS value#9, 'key AS col2#10]
   +- 'UnresolvedRelation `src`

== Analyzed Logical Plan ==
value: int, col2: int
Project [value#9, col2#10]
+- Sort [value#9 DESC NULLS LAST, col2#10 DESC NULLS LAST], true
   +- Aggregate [key#5], [max(value#6) AS value#9, key#5 AS col2#10]
      +- SubqueryAlias src
         +- Project [_1#2 AS key#5, _2#3 AS value#6]
            +- LocalRelation [_1#2, _2#3]
``` SQL
The sort direction is being wrongly changed from ASC to DSC while resolving ```Sort``` in
resolveAggregateFunctions.

The above testcase models TPCDS-Q71 and thus we have the same issue in Q71 as well.

## How was this patch tested?
A few tests are added in SQLQuerySuite.

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

Closes #20453 from dilipbiswal/local_spark.
2018-01-31 13:52:47 -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
gatorsmile 31c00ad8b0 [SPARK-23267][SQL] Increase spark.sql.codegen.hugeMethodLimit to 65535
## What changes were proposed in this pull request?
Still saw the performance regression introduced by `spark.sql.codegen.hugeMethodLimit` in our internal workloads. There are two major issues in the current solution.
- The size of the complied byte code is not identical to the bytecode size of the method. The detection is still not accurate.
- The bytecode size of a single operator (e.g., `SerializeFromObject`) could still exceed 8K limit. We saw the performance regression in such scenario.

Since it is close to the release of 2.3, we decide to increase it to 64K for avoiding the perf regression.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20434 from gatorsmile/revertConf.
2018-01-30 11:33:30 -08:00
gatorsmile 7a2ada223e [SPARK-23261][PYSPARK] Rename Pandas UDFs
## What changes were proposed in this pull request?
Rename the public APIs and names of pandas udfs.

- `PANDAS SCALAR UDF` -> `SCALAR PANDAS UDF`
- `PANDAS GROUP MAP UDF` -> `GROUPED MAP PANDAS UDF`
- `PANDAS GROUP AGG UDF` -> `GROUPED AGG PANDAS UDF`

## How was this patch tested?
The existing tests

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20428 from gatorsmile/renamePandasUDFs.
2018-01-30 21:55:55 +09:00
Bryan Cutler f235df66a4 [SPARK-22221][SQL][FOLLOWUP] Externalize spark.sql.execution.arrow.maxRecordsPerBatch
## What changes were proposed in this pull request?

This is a followup to #19575 which added a section on setting max Arrow record batches and this will externalize the conf that was referenced in the docs.

## How was this patch tested?
NA

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #20423 from BryanCutler/arrow-user-doc-externalize-maxRecordsPerBatch-SPARK-22221.
2018-01-29 17:37:55 -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
Wang Gengliang badf0d0e0d [SPARK-23219][SQL] Rename ReadTask to DataReaderFactory in data source v2
## What changes were proposed in this pull request?

Currently we have `ReadTask` in data source v2 reader, while in writer we have `DataWriterFactory`.
To make the naming consistent and better, renaming `ReadTask` to `DataReaderFactory`.

## How was this patch tested?

Unit test

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #20397 from gengliangwang/rename.
2018-01-30 00:50:49 +08:00
hyukjinkwon 39d2c6b034 [SPARK-23238][SQL] Externalize SQLConf configurations exposed in documentation
## What changes were proposed in this pull request?

This PR proposes to expose few internal configurations found in the documentation.

Also it fixes the description for `spark.sql.execution.arrow.enabled`.
It's quite self-explanatory.

## How was this patch tested?

N/A

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20403 from HyukjinKwon/minor-doc-arrow.
2018-01-29 21:09:05 +09:00
Jose Torres 49b0207dc9 [SPARK-23196] Unify continuous and microbatch V2 sinks
## What changes were proposed in this pull request?

Replace streaming V2 sinks with a unified StreamWriteSupport interface, with a shim to use it with microbatch execution.

Add a new SQL config to use for disabling V2 sinks, falling back to the V1 sink implementation.

## How was this patch tested?

Existing tests, which in the case of Kafka (the only existing continuous V2 sink) now use V2 for microbatch.

Author: Jose Torres <jose@databricks.com>

Closes #20369 from jose-torres/streaming-sink.
2018-01-29 13:10:38 +08:00
Xingbo Jiang 94c67a76ec [SPARK-23207][SQL] Shuffle+Repartition on a DataFrame could lead to incorrect answers
## What changes were proposed in this pull request?

Currently shuffle repartition uses RoundRobinPartitioning, the generated result is nondeterministic since the sequence of input rows are not determined.

The bug can be triggered when there is a repartition call following a shuffle (which would lead to non-deterministic row ordering), as the pattern shows below:
upstream stage -> repartition stage -> result stage
(-> indicate a shuffle)
When one of the executors process goes down, some tasks on the repartition stage will be retried and generate inconsistent ordering, and some tasks of the result stage will be retried generating different data.

The following code returns 931532, instead of 1000000:
```
import scala.sys.process._

import org.apache.spark.TaskContext
val res = spark.range(0, 1000 * 1000, 1).repartition(200).map { x =>
  x
}.repartition(200).map { x =>
  if (TaskContext.get.attemptNumber == 0 && TaskContext.get.partitionId < 2) {
    throw new Exception("pkill -f java".!!)
  }
  x
}
res.distinct().count()
```

In this PR, we propose a most straight-forward way to fix this problem by performing a local sort before partitioning, after we make the input row ordering deterministic, the function from rows to partitions is fully deterministic too.

The downside of the approach is that with extra local sort inserted, the performance of repartition() will go down, so we add a new config named `spark.sql.execution.sortBeforeRepartition` to control whether this patch is applied. The patch is default enabled to be safe-by-default, but user may choose to manually turn it off to avoid performance regression.

This patch also changes the output rows ordering of repartition(), that leads to a bunch of test cases failure because they are comparing the results directly.

## How was this patch tested?

Add unit test in ExchangeSuite.

With this patch(and `spark.sql.execution.sortBeforeRepartition` set to true), the following query returns 1000000:
```
import scala.sys.process._

import org.apache.spark.TaskContext

spark.conf.set("spark.sql.execution.sortBeforeRepartition", "true")

val res = spark.range(0, 1000 * 1000, 1).repartition(200).map { x =>
  x
}.repartition(200).map { x =>
  if (TaskContext.get.attemptNumber == 0 && TaskContext.get.partitionId < 2) {
    throw new Exception("pkill -f java".!!)
  }
  x
}
res.distinct().count()

res7: Long = 1000000
```

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

Closes #20393 from jiangxb1987/shuffle-repartition.
2018-01-26 15:01:03 -08:00
Kris Mok e57f394818 [SPARK-23032][SQL] Add a per-query codegenStageId to WholeStageCodegenExec
## What changes were proposed in this pull request?

**Proposal**

Add a per-query ID to the codegen stages as represented by `WholeStageCodegenExec` operators. This ID will be used in
-  the explain output of the physical plan, and in
- the generated class name.

Specifically, this ID will be stable within a query, counting up from 1 in depth-first post-order for all the `WholeStageCodegenExec` inserted into a plan.
The ID value 0 is reserved for "free-floating" `WholeStageCodegenExec` objects, which may have been created for one-off purposes, e.g. for fallback handling of codegen stages that failed to codegen the whole stage and wishes to codegen a subset of the children operators (as seen in `org.apache.spark.sql.execution.FileSourceScanExec#doExecute`).

Example: for the following query:
```scala
scala> spark.conf.set("spark.sql.autoBroadcastJoinThreshold", 1)

scala> val df1 = spark.range(10).select('id as 'x, 'id + 1 as 'y).orderBy('x).select('x + 1 as 'z, 'y)
df1: org.apache.spark.sql.DataFrame = [z: bigint, y: bigint]

scala> val df2 = spark.range(5)
df2: org.apache.spark.sql.Dataset[Long] = [id: bigint]

scala> val query = df1.join(df2, 'z === 'id)
query: org.apache.spark.sql.DataFrame = [z: bigint, y: bigint ... 1 more field]
```

The explain output before the change is:
```scala
scala> query.explain
== Physical Plan ==
*SortMergeJoin [z#9L], [id#13L], Inner
:- *Sort [z#9L ASC NULLS FIRST], false, 0
:  +- Exchange hashpartitioning(z#9L, 200)
:     +- *Project [(x#3L + 1) AS z#9L, y#4L]
:        +- *Sort [x#3L ASC NULLS FIRST], true, 0
:           +- Exchange rangepartitioning(x#3L ASC NULLS FIRST, 200)
:              +- *Project [id#0L AS x#3L, (id#0L + 1) AS y#4L]
:                 +- *Range (0, 10, step=1, splits=8)
+- *Sort [id#13L ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(id#13L, 200)
      +- *Range (0, 5, step=1, splits=8)
```
Note how codegen'd operators are annotated with a prefix `"*"`. See how the `SortMergeJoin` operator and its direct children `Sort` operators are adjacent and all annotated with the `"*"`, so it's hard to tell they're actually in separate codegen stages.

and after this change it'll be:
```scala
scala> query.explain
== Physical Plan ==
*(6) SortMergeJoin [z#9L], [id#13L], Inner
:- *(3) Sort [z#9L ASC NULLS FIRST], false, 0
:  +- Exchange hashpartitioning(z#9L, 200)
:     +- *(2) Project [(x#3L + 1) AS z#9L, y#4L]
:        +- *(2) Sort [x#3L ASC NULLS FIRST], true, 0
:           +- Exchange rangepartitioning(x#3L ASC NULLS FIRST, 200)
:              +- *(1) Project [id#0L AS x#3L, (id#0L + 1) AS y#4L]
:                 +- *(1) Range (0, 10, step=1, splits=8)
+- *(5) Sort [id#13L ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(id#13L, 200)
      +- *(4) Range (0, 5, step=1, splits=8)
```
Note that the annotated prefix becomes `"*(id) "`. See how the `SortMergeJoin` operator and its direct children `Sort` operators have different codegen stage IDs.

It'll also show up in the name of the generated class, as a suffix in the format of `GeneratedClass$GeneratedIterator$id`.

For example, note how `GeneratedClass$GeneratedIteratorForCodegenStage3` and `GeneratedClass$GeneratedIteratorForCodegenStage6` in the following stack trace corresponds to the IDs shown in the explain output above:
```
"Executor task launch worker for task 42412957" daemon prio=5 tid=0x58 nid=NA runnable
  java.lang.Thread.State: RUNNABLE
	  at org.apache.spark.sql.execution.UnsafeExternalRowSorter.insertRow(UnsafeExternalRowSorter.java:109)
	  at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage3.sort_addToSorter$(generated.java:32)
	  at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage3.processNext(generated.java:41)
	  at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	  at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$9$$anon$1.hasNext(WholeStageCodegenExec.scala:494)
	  at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage6.findNextInnerJoinRows$(generated.java:42)
	  at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage6.processNext(generated.java:101)
	  at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	  at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$11$$anon$2.hasNext(WholeStageCodegenExec.scala:513)
	  at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:253)
	  at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:247)
	  at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:828)
	  at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:828)
	  at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	  at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
	  at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
	  at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
	  at org.apache.spark.scheduler.Task.run(Task.scala:109)
	  at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345)
	  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	  at java.lang.Thread.run(Thread.java:748)
```

**Rationale**

Right now, the codegen from Spark SQL lacks the means to differentiate between a couple of things:

1. It's hard to tell which physical operators are in the same WholeStageCodegen stage. Note that this "stage" is a separate notion from Spark's RDD execution stages; this one is only to delineate codegen units.
There can be adjacent physical operators that are both codegen'd but are in separate codegen stages. Some of this is due to hacky implementation details, such as the case with `SortMergeJoin` and its `Sort` inputs -- they're hard coded to be split into separate stages although both are codegen'd.
When printing out the explain output of the physical plan, you'd only see the codegen'd physical operators annotated with a preceding star (`'*'`) but would have no way to figure out if they're in the same stage.

2. Performance/error diagnosis
The generated code has class/method names that are hard to differentiate between queries or even between codegen stages within the same query. If we use a Java-level profiler to collect profiles, or if we encounter a Java-level exception with a stack trace in it, it's really hard to tell which part of a query it's at.
By introducing a per-query codegen stage ID, we'd at least be able to know which codegen stage (and in turn, which group of physical operators) was a profile tick or an exception happened.

The reason why this proposal uses a per-query ID is because it's stable within a query, so that multiple runs of the same query will see the same resulting IDs. This both benefits understandability for users, and also it plays well with the codegen cache in Spark SQL which uses the generated source code as the key.

The downside to using per-query IDs as opposed to a per-session or globally incrementing ID is of course we can't tell apart different query runs with this ID alone. But for now I believe this is a good enough tradeoff.

## How was this patch tested?

Existing tests. This PR does not involve any runtime behavior changes other than some name changes.
The SQL query test suites that compares explain outputs have been updates to ignore the newly added `codegenStageId`.

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

Closes #20224 from rednaxelafx/wsc-codegenstageid.
2018-01-25 16:11:33 -08:00
Liang-Chi Hsieh d20bbc2d87 [SPARK-21717][SQL] Decouple consume functions of physical operators in whole-stage codegen
## What changes were proposed in this pull request?

It has been observed in SPARK-21603 that whole-stage codegen suffers performance degradation, if the generated functions are too long to be optimized by JIT.

We basically produce a single function to incorporate generated codes from all physical operators in whole-stage. Thus, it is possibly to grow the size of generated function over a threshold that we can't have JIT optimization for it anymore.

This patch is trying to decouple the logic of consuming rows in physical operators to avoid a giant function processing rows.

## How was this patch tested?

Added tests.

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

Closes #18931 from viirya/SPARK-21717.
2018-01-25 19:49:58 +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
gatorsmile ee572ba8c1 [SPARK-20749][SQL][FOLLOW-UP] Override prettyName for bit_length and octet_length
## What changes were proposed in this pull request?
We need to override the prettyName for bit_length and octet_length for getting the expected auto-generated alias name.

## How was this patch tested?
The existing tests

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20358 from gatorsmile/test2.3More.
2018-01-23 21:36:20 +09:00
Li Jin b2ce17b4c9 [SPARK-22274][PYTHON][SQL] User-defined aggregation functions with pandas udf (full shuffle)
## What changes were proposed in this pull request?

Add support for using pandas UDFs with groupby().agg().

This PR introduces a new type of pandas UDF - group aggregate pandas UDF. This type of UDF defines a transformation of multiple pandas Series -> a scalar value. Group aggregate pandas UDFs can be used with groupby().agg(). Note group aggregate pandas UDF doesn't support partial aggregation, i.e., a full shuffle is required.

This PR doesn't support group aggregate pandas UDFs that return ArrayType, StructType or MapType. Support for these types is left for future PR.

## How was this patch tested?

GroupbyAggPandasUDFTests

Author: Li Jin <ice.xelloss@gmail.com>

Closes #19872 from icexelloss/SPARK-22274-groupby-agg.
2018-01-23 14:11:30 +09:00
Wenchen Fan 51eb750263 [SPARK-22389][SQL] data source v2 partitioning reporting interface
## What changes were proposed in this pull request?

a new interface which allows data source to report partitioning and avoid shuffle at Spark side.

The design is pretty like the internal distribution/partitioing framework. Spark defines a `Distribution` interfaces and several concrete implementations, and ask the data source to report a `Partitioning`, the `Partitioning` should tell Spark if it can satisfy a `Distribution` or not.

## How was this patch tested?

new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20201 from cloud-fan/partition-reporting.
2018-01-22 15:21: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
Wenchen Fan 5d680cae48 [SPARK-23090][SQL] polish ColumnVector
## What changes were proposed in this pull request?

Several improvements:
* provide a default implementation for the batch get methods
* rename `getChildColumn` to `getChild`, which is more concise
* remove `getStruct(int, int)`, it's only used to simplify the codegen, which is an internal thing, we should not add a public API for this purpose.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20277 from cloud-fan/column-vector.
2018-01-22 20:56:38 +08:00
gatorsmile 78801881c4 [SPARK-23170][SQL] Dump the statistics of effective runs of analyzer and optimizer rules
## What changes were proposed in this pull request?

Dump the statistics of effective runs of analyzer and optimizer rules.

## How was this patch tested?

Do a manual run of TPCDSQuerySuite

```
=== Metrics of Analyzer/Optimizer Rules ===
Total number of runs: 175899
Total time: 25.486559948 seconds

Rule                                                                                               Effective Time / Total Time                     Effective Runs / Total Runs

org.apache.spark.sql.catalyst.optimizer.ColumnPruning                                              1603280450 / 2868461549                         761 / 1877
org.apache.spark.sql.catalyst.analysis.Analyzer$CTESubstitution                                    2045860009 / 2056602674                         37 / 788
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAggregateFunctions                          440719059 / 1693110949                          38 / 1982
org.apache.spark.sql.catalyst.optimizer.Optimizer$OptimizeSubqueries                               1429834919 / 1446016225                         39 / 285
org.apache.spark.sql.catalyst.optimizer.PruneFilters                                               33273083 / 1389586938                           3 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences                                  821183615 / 1266668754                          616 / 1982
org.apache.spark.sql.catalyst.optimizer.ReorderJoin                                                775837028 / 866238225                           132 / 1592
org.apache.spark.sql.catalyst.analysis.DecimalPrecision                                            550683593 / 748854507                           211 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveSubquery                                    513075345 / 634370596                           49 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$FixNullability                                     33475731 / 606406532                            12 / 742
org.apache.spark.sql.catalyst.analysis.TypeCoercion$ImplicitTypeCasts                              193144298 / 545403925                           86 / 1982
org.apache.spark.sql.catalyst.optimizer.BooleanSimplification                                      18651497 / 495725004                            7 / 1592
org.apache.spark.sql.catalyst.optimizer.PushPredicateThroughJoin                                   369257217 / 489934378                           709 / 1592
org.apache.spark.sql.catalyst.optimizer.RemoveRedundantAliases                                     3707000 / 468291609                             9 / 1592
org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints                                410155900 / 435254175                           192 / 285
org.apache.spark.sql.execution.datasources.FindDataSourceTable                                     348885539 / 371855866                           233 / 1982
org.apache.spark.sql.catalyst.optimizer.NullPropagation                                            11307645 / 307531225                            26 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveFunctions                                   120324545 / 304948785                           294 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$FunctionArgumentConversion                     92323199 / 286695007                            38 / 1982
org.apache.spark.sql.catalyst.optimizer.PushDownPredicate                                          230084193 / 265845972                           785 / 1592
org.apache.spark.sql.catalyst.analysis.TypeCoercion$PromoteStrings                                 45938401 / 265144009                            40 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$InConversion                                   14888776 / 261499450                            1 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$CaseWhenCoercion                               113796384 / 244913861                           29 / 1982
org.apache.spark.sql.catalyst.optimizer.ConstantFolding                                            65008069 / 236548480                            126 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ExtractGenerator                                   0 / 226338929                                   0 / 1982
org.apache.spark.sql.catalyst.analysis.ResolveTimeZone                                             98134906 / 221323770                            417 / 1982
org.apache.spark.sql.catalyst.optimizer.ReorderAssociativeOperator                                 0 / 208421703                                   0 / 1592
org.apache.spark.sql.catalyst.optimizer.OptimizeIn                                                 8762534 / 199351958                             16 / 1592
org.apache.spark.sql.catalyst.analysis.TypeCoercion$DateTimeOperations                             11980016 / 190779046                            27 / 1982
org.apache.spark.sql.catalyst.optimizer.SimplifyBinaryComparison                                   0 / 188887385                                   0 / 1592
org.apache.spark.sql.catalyst.optimizer.SimplifyConditionals                                       0 / 186812106                                   0 / 1592
org.apache.spark.sql.catalyst.optimizer.SimplifyCaseConversionExpressions                          0 / 183885230                                   0 / 1592
org.apache.spark.sql.catalyst.optimizer.SimplifyCasts                                              17128295 / 182901910                            69 / 1592
org.apache.spark.sql.catalyst.analysis.TypeCoercion$Division                                       14579110 / 180309340                            8 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$BooleanEquality                                0 / 176740516                                   0 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$IfCoercion                                     0 / 170781986                                   0 / 1982
org.apache.spark.sql.catalyst.optimizer.LikeSimplification                                         771605 / 164136736                              1 / 1592
org.apache.spark.sql.catalyst.optimizer.RemoveDispensableExpressions                               0 / 155958962                                   0 / 1592
org.apache.spark.sql.catalyst.analysis.ResolveCreateNamedStruct                                    0 / 151222943                                   0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveWindowOrder                                 7534632 / 146596355                             14 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$EltCoercion                                    0 / 144488654                                   0 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$ConcatCoercion                                 0 / 142403338                                   0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveWindowFrame                                 12067635 / 141500665                            21 / 1982
org.apache.spark.sql.catalyst.analysis.TimeWindowing                                               0 / 140431958                                   0 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$WindowFrameCoercion                            0 / 125471960                                   0 / 1982
org.apache.spark.sql.catalyst.optimizer.EliminateOuterJoin                                         14226972 / 124922019                            11 / 1592
org.apache.spark.sql.catalyst.analysis.TypeCoercion$StackCoercion                                  0 / 123613887                                   0 / 1982
org.apache.spark.sql.catalyst.optimizer.RewriteCorrelatedScalarSubquery                            8491071 / 121179056                             7 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveGroupingAnalytics                           55526073 / 120290529                            11 / 1982
org.apache.spark.sql.catalyst.optimizer.ConstantPropagation                                        0 / 113886790                                   0 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveDeserializer                                52383759 / 107160222                            148 / 1982
org.apache.spark.sql.catalyst.analysis.CleanupAliases                                              52543524 / 102091518                            344 / 1086
org.apache.spark.sql.catalyst.optimizer.RemoveRedundantProject                                     40682895 / 94403652                             342 / 1877
org.apache.spark.sql.catalyst.analysis.Analyzer$ExtractWindowExpressions                           38473816 / 89740578                             23 / 1982
org.apache.spark.sql.catalyst.optimizer.CollapseProject                                            46806090 / 83315506                             281 / 1877
org.apache.spark.sql.catalyst.optimizer.FoldablePropagation                                        0 / 78750087                                    0 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAliases                                     13742765 / 77227258                             47 / 1982
org.apache.spark.sql.catalyst.optimizer.CombineFilters                                             53386729 / 76960344                             448 / 1592
org.apache.spark.sql.execution.datasources.DataSourceAnalysis                                      68034341 / 75724186                             24 / 742
org.apache.spark.sql.catalyst.analysis.Analyzer$LookupFunctions                                    0 / 71151084                                    0 / 750
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveMissingReferences                           12139848 / 67599140                             8 / 1982
org.apache.spark.sql.catalyst.optimizer.PullupCorrelatedPredicates                                 45017938 / 65968777                             23 / 285
org.apache.spark.sql.execution.datasources.v2.PushDownOperatorsToDataSource                        0 / 60937767                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.CollapseRepartition                                        0 / 59897237                                    0 / 1592
org.apache.spark.sql.catalyst.optimizer.PushProjectionThroughUnion                                 8547262 / 53941370                              10 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$HandleNullInputsForUDF                             0 / 52735976                                    0 / 742
org.apache.spark.sql.catalyst.analysis.TypeCoercion$WidenSetOperationTypes                         9797713 / 52401665                              9 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$PullOutNondeterministic                            0 / 51741500                                    0 / 742
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations                                   28614911 / 51061186                             233 / 1990
org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions                               0 / 50621510                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.CombineUnions                                              2777800 / 50262112                              17 / 1877
org.apache.spark.sql.catalyst.analysis.Analyzer$GlobalAggregates                                   1640641 / 49633909                              46 / 1982
org.apache.spark.sql.catalyst.optimizer.DecimalAggregates                                          20198374 / 48488419                             100 / 385
org.apache.spark.sql.catalyst.optimizer.LimitPushDown                                              0 / 45052523                                    0 / 1592
org.apache.spark.sql.catalyst.optimizer.CombineLimits                                              0 / 44719443                                    0 / 1592
org.apache.spark.sql.catalyst.optimizer.EliminateSorts                                             0 / 44216930                                    0 / 1592
org.apache.spark.sql.catalyst.optimizer.RewritePredicateSubquery                                   36235699 / 44165786                             148 / 285
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveNewInstance                                 0 / 42750307                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveUpCast                                      0 / 41811748                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveOrdinalInOrderByAndGroupBy                  3819476 / 41776562                              4 / 1982
org.apache.spark.sql.catalyst.optimizer.ComputeCurrentTime                                         0 / 40527808                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.CollapseWindow                                             0 / 36832538                                    0 / 1592
org.apache.spark.sql.catalyst.optimizer.EliminateSerialization                                     0 / 36120667                                    0 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAggAliasInGroupBy                           0 / 32435826                                    0 / 1982
org.apache.spark.sql.execution.datasources.PreprocessTableCreation                                 0 / 32145218                                    0 / 742
org.apache.spark.sql.execution.datasources.ResolveSQLOnFile                                        0 / 30295614                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolvePivot                                       0 / 30111655                                    0 / 1982
org.apache.spark.sql.catalyst.expressions.codegen.package$ExpressionCanonicalizer$CleanExpressions 59930 / 28038201                                26 / 8280
org.apache.spark.sql.catalyst.analysis.ResolveInlineTables                                         0 / 27808108                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveSubqueryColumnAliases                       0 / 27066690                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveGenerate                                    0 / 26660210                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveNaturalAndUsingJoin                         0 / 25255184                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.ResolveTableValuedFunctions                                 0 / 24663088                                    0 / 1990
org.apache.spark.sql.catalyst.analysis.SubstituteUnresolvedOrdinals                                9709079 / 24450670                              4 / 788
org.apache.spark.sql.catalyst.analysis.ResolveHints$ResolveBroadcastHints                          0 / 23776535                                    0 / 750
org.apache.spark.sql.catalyst.optimizer.ReplaceExpressions                                         0 / 22697895                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.CheckCartesianProducts                                     0 / 22523798                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.ReplaceDistinctWithAggregate                               988593 / 21535410                               15 / 300
org.apache.spark.sql.catalyst.optimizer.EliminateMapObjects                                        0 / 20269996                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.RewriteDistinctAggregates                                  0 / 19388592                                    0 / 285
org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases                                    17675532 / 18971185                             215 / 285
org.apache.spark.sql.catalyst.optimizer.GetCurrentDatabase                                         0 / 18271152                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.PropagateEmptyRelation                                     2077097 / 17190855                              3 / 288
org.apache.spark.sql.catalyst.analysis.EliminateBarriers                                           0 / 16736359                                    0 / 1086
org.apache.spark.sql.execution.OptimizeMetadataOnlyQuery                                           0 / 16669341                                    0 / 285
org.apache.spark.sql.catalyst.analysis.UpdateOuterReferences                                       0 / 14470235                                    0 / 742
org.apache.spark.sql.catalyst.optimizer.ReplaceExceptWithAntiJoin                                  6715625 / 12190561                              1 / 300
org.apache.spark.sql.catalyst.optimizer.ReplaceIntersectWithSemiJoin                               3451793 / 11431432                              7 / 300
org.apache.spark.sql.execution.python.ExtractPythonUDFFromAggregate                                0 / 10810568                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.RemoveRepetitionFromGroupExpressions                       344198 / 10475276                               1 / 286
org.apache.spark.sql.catalyst.analysis.Analyzer$WindowsSubstitution                                0 / 10386630                                    0 / 788
org.apache.spark.sql.catalyst.analysis.EliminateUnions                                             0 / 10096526                                    0 / 788
org.apache.spark.sql.catalyst.analysis.AliasViewChild                                              0 / 9991706                                     0 / 742
org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation                                     0 / 9649334                                     0 / 288
org.apache.spark.sql.catalyst.analysis.ResolveHints$RemoveAllHints                                 0 / 8739109                                     0 / 750
org.apache.spark.sql.execution.datasources.PreprocessTableInsertion                                0 / 8420889                                     0 / 742
org.apache.spark.sql.catalyst.analysis.EliminateView                                               0 / 8319134                                     0 / 285
org.apache.spark.sql.catalyst.optimizer.RemoveLiteralFromGroupExpressions                          0 / 7392627                                     0 / 286
org.apache.spark.sql.catalyst.optimizer.ReplaceExceptWithFilter                                    0 / 7170516                                     0 / 300
org.apache.spark.sql.catalyst.optimizer.SimplifyCreateArrayOps                                     0 / 7109643                                     0 / 1592
org.apache.spark.sql.catalyst.optimizer.SimplifyCreateStructOps                                    0 / 6837590                                     0 / 1592
org.apache.spark.sql.catalyst.optimizer.SimplifyCreateMapOps                                       0 / 6617848                                     0 / 1592
org.apache.spark.sql.catalyst.optimizer.CombineConcats                                             0 / 5768406                                     0 / 1592
org.apache.spark.sql.catalyst.optimizer.ReplaceDeduplicateWithAggregate                            0 / 5349831                                     0 / 285
org.apache.spark.sql.catalyst.optimizer.CombineTypedFilters                                        0 / 5186642                                     0 / 285
org.apache.spark.sql.catalyst.optimizer.EliminateDistinct                                          0 / 2427686                                     0 / 285
org.apache.spark.sql.catalyst.optimizer.CostBasedJoinReorder                                       0 / 2420436                                     0 / 285

```

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20342 from gatorsmile/reportExecution.
2018-01-22 04:31:24 -08:00
Marco Gaido 121dc96f08 [SPARK-23087][SQL] CheckCartesianProduct too restrictive when condition is false/null
## What changes were proposed in this pull request?

CheckCartesianProduct raises an AnalysisException also when the join condition is always false/null. In this case, we shouldn't raise it, since the result will not be a cartesian product.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20333 from mgaido91/SPARK-23087.
2018-01-20 22:39:49 -08:00
Takuya UESHIN 568055da93 [SPARK-23054][SQL][PYSPARK][FOLLOWUP] Use sqlType casting when casting PythonUserDefinedType to String.
## What changes were proposed in this pull request?

This is a follow-up of #20246.

If a UDT in Python doesn't have its corresponding Scala UDT, cast to string will be the raw string of the internal value, e.g. `"org.apache.spark.sql.catalyst.expressions.UnsafeArrayDataxxxxxxxx"` if the internal type is `ArrayType`.

This pr fixes it by using its `sqlType` casting.

## How was this patch tested?

Added a test and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #20306 from ueshin/issues/SPARK-23054/fup1.
2018-01-19 11:37:08 +08:00
Burak Yavuz e01919e834 [SPARK-23094] Fix invalid character handling in JsonDataSource
## What changes were proposed in this pull request?

There were two related fixes regarding `from_json`, `get_json_object` and `json_tuple` ([Fix #1](c8803c0685),
 [Fix #2](86174ea89b)), but they weren't comprehensive it seems. I wanted to extend those fixes to all the parsers, and add tests for each case.

## How was this patch tested?

Regression tests

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #20302 from brkyvz/json-invfix.
2018-01-19 07:36:06 +09: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
Dongjoon Hyun 0f8a28617a [SPARK-21783][SQL] Turn on ORC filter push-down by default
## What changes were proposed in this pull request?

ORC filter push-down is disabled by default from the beginning, [SPARK-2883](aa31e431fc (diff-41ef65b9ef5b518f77e2a03559893f4dR149)
).

Now, Apache Spark starts to depend on Apache ORC 1.4.1. For Apache Spark 2.3, this PR turns on ORC filter push-down by default like Parquet ([SPARK-9207](https://issues.apache.org/jira/browse/SPARK-21783)) as a part of [SPARK-20901](https://issues.apache.org/jira/browse/SPARK-20901), "Feature parity for ORC with Parquet".

## How was this patch tested?

Pass the existing tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20265 from dongjoon-hyun/SPARK-21783.
2018-01-17 21:53:36 +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
Dilip Biswal 0c2ba427bc [SPARK-23095][SQL] Decorrelation of scalar subquery fails with java.util.NoSuchElementException
## What changes were proposed in this pull request?
The following SQL involving scalar correlated query returns a map exception.
``` SQL
SELECT t1a
FROM   t1
WHERE  t1a = (SELECT   count(*)
              FROM     t2
              WHERE    t2c = t1c
              HAVING   count(*) >= 1)
```
``` SQL
key not found: ExprId(278,786682bb-41f9-4bd5-a397-928272cc8e4e)
java.util.NoSuchElementException: key not found: ExprId(278,786682bb-41f9-4bd5-a397-928272cc8e4e)
        at scala.collection.MapLike$class.default(MapLike.scala:228)
        at scala.collection.AbstractMap.default(Map.scala:59)
        at scala.collection.MapLike$class.apply(MapLike.scala:141)
        at scala.collection.AbstractMap.apply(Map.scala:59)
        at org.apache.spark.sql.catalyst.optimizer.RewriteCorrelatedScalarSubquery$.org$apache$spark$sql$catalyst$optimizer$RewriteCorrelatedScalarSubquery$$evalSubqueryOnZeroTups(subquery.scala:378)
        at org.apache.spark.sql.catalyst.optimizer.RewriteCorrelatedScalarSubquery$$anonfun$org$apache$spark$sql$catalyst$optimizer$RewriteCorrelatedScalarSubquery$$constructLeftJoins$1.apply(subquery.scala:430)
        at org.apache.spark.sql.catalyst.optimizer.RewriteCorrelatedScalarSubquery$$anonfun$org$apache$spark$sql$catalyst$optimizer$RewriteCorrelatedScalarSubquery$$constructLeftJoins$1.apply(subquery.scala:426)
```

In this case, after evaluating the HAVING clause "count(*) > 1" statically
against the binding of aggregtation result on empty input, we determine
that this query will not have a the count bug. We should simply return
the evalSubqueryOnZeroTups with empty value.
(Please fill in changes proposed in this fix)

## How was this patch tested?
A new test was added in the Subquery bucket.

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

Closes #20283 from dilipbiswal/scalar-count-defect.
2018-01-17 09:57:30 +08:00
Marco Gaido 8ab2d7ea99 [SPARK-23080][SQL] Improve error message for built-in functions
## What changes were proposed in this pull request?

When a user puts the wrong number of parameters in a function, an AnalysisException is thrown. If the function is a UDF, he user is told how many parameters the function expected and how many he/she put. If the function, instead, is a built-in one, no information about the number of parameters expected and the actual one is provided. This can help in some cases, to debug the errors (eg. bad quotes escaping may lead to a different number of parameters than expected, etc. etc.)

The PR adds the information about the number of parameters passed and the expected one, analogously to what happens for UDF.

## How was this patch tested?

modified existing UT + manual test

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20271 from mgaido91/SPARK-23080.
2018-01-16 11:47:42 +09: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
Yuming Wang a38c887ac0 [SPARK-19550][BUILD][FOLLOW-UP] Remove MaxPermSize for sql module
## What changes were proposed in this pull request?

Remove `MaxPermSize` for `sql` module

## How was this patch tested?

Manually tested.

Author: Yuming Wang <yumwang@ebay.com>

Closes #20268 from wangyum/SPARK-19550-MaxPermSize.
2018-01-15 07:49:34 -06:00
Takeshi Yamamuro b98ffa4d6d [SPARK-23054][SQL] Fix incorrect results of casting UserDefinedType to String
## What changes were proposed in this pull request?
This pr fixed the issue when casting `UserDefinedType`s into strings;
```
>>> from pyspark.ml.classification import MultilayerPerceptronClassifier
>>> from pyspark.ml.linalg import Vectors
>>> df = spark.createDataFrame([(0.0, Vectors.dense([0.0, 0.0])), (1.0, Vectors.dense([0.0, 1.0]))], ["label", "features"])
>>> df.selectExpr("CAST(features AS STRING)").show(truncate = False)
+-------------------------------------------+
|features                                   |
+-------------------------------------------+
|[6,1,0,0,2800000020,2,0,0,0]               |
|[6,1,0,0,2800000020,2,0,0,3ff0000000000000]|
+-------------------------------------------+
```
The root cause is that `Cast` handles input data as `UserDefinedType.sqlType`(this is underlying storage type), so we should pass data into `UserDefinedType.deserialize` then `toString`.
This pr modified the result into;
```
+---------+
|features |
+---------+
|[0.0,0.0]|
|[0.0,1.0]|
+---------+
```

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20246 from maropu/SPARK-23054.
2018-01-15 10:55:21 +08:00
guoxiaolong 42a1a15d73 [SPARK-22999][SQL] show databases like command' can remove the like keyword
## What changes were proposed in this pull request?

SHOW DATABASES (LIKE pattern = STRING)? Can be like the back increase?
When using this command, LIKE keyword can be removed.
You can refer to the SHOW TABLES command, SHOW TABLES 'test *' and SHOW TABELS like 'test *' can be used.
Similarly SHOW DATABASES 'test *' and SHOW DATABASES like 'test *' can be used.

## How was this patch tested?
unit tests   manual tests
Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: guoxiaolong <guo.xiaolong1@zte.com.cn>

Closes #20194 from guoxiaolongzte/SPARK-22999.
2018-01-15 02:02:49 +08:00
Takeshi Yamamuro 990f05c803 [SPARK-23021][SQL] AnalysisBarrier should override innerChildren to print correct explain output
## What changes were proposed in this pull request?
`AnalysisBarrier` in the current master cuts off explain results for parsed logical plans;
```
scala> Seq((1, 1)).toDF("a", "b").groupBy("a").count().sample(0.1).explain(true)
== Parsed Logical Plan ==
Sample 0.0, 0.1, false, -7661439431999668039
+- AnalysisBarrier Aggregate [a#5], [a#5, count(1) AS count#14L]
```
To fix this, `AnalysisBarrier` needs to override `innerChildren` and this pr changed the output to;
```
== Parsed Logical Plan ==
Sample 0.0, 0.1, false, -5086223488015741426
+- AnalysisBarrier
      +- Aggregate [a#5], [a#5, count(1) AS count#14L]
         +- Project [_1#2 AS a#5, _2#3 AS b#6]
            +- LocalRelation [_1#2, _2#3]
```

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20247 from maropu/SPARK-23021-2.
2018-01-14 22:26:21 +08:00
CodingCat ba891ec993 [SPARK-22790][SQL] add a configurable factor to describe HadoopFsRelation's size
## What changes were proposed in this pull request?

as per discussion in https://github.com/apache/spark/pull/19864#discussion_r156847927

the current HadoopFsRelation is purely based on the underlying file size which is not accurate and makes the execution vulnerable to errors like OOM

Users can enable CBO with the functionalities in https://github.com/apache/spark/pull/19864 to avoid this issue

This JIRA proposes to add a configurable factor to sizeInBytes method in HadoopFsRelation class so that users can mitigate this problem without CBO

## How was this patch tested?

Existing tests

Author: CodingCat <zhunansjtu@gmail.com>
Author: Nan Zhu <nanzhu@uber.com>

Closes #20072 from CodingCat/SPARK-22790.
2018-01-14 02:36:32 +08:00
gatorsmile 651f76153f [SPARK-23028] Bump master branch version to 2.4.0-SNAPSHOT
## What changes were proposed in this pull request?
This patch bumps the master branch version to `2.4.0-SNAPSHOT`.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20222 from gatorsmile/bump24.
2018-01-13 00:37:59 +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
Wenchen Fan eaac60a1e2 [SPARK-16060][SQL][FOLLOW-UP] add a wrapper solution for vectorized orc reader
## What changes were proposed in this pull request?

This is mostly from https://github.com/apache/spark/pull/13775

The wrapper solution is pretty good for string/binary type, as the ORC column vector doesn't keep bytes in a continuous memory region, and has a significant overhead when copying the data to Spark columnar batch. For other cases, the wrapper solution is almost same with the current solution.

I think we can treat the wrapper solution as a baseline and keep improving the writing to Spark solution.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20205 from cloud-fan/orc.
2018-01-10 15:16:27 +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
Dongjoon Hyun f44ba910f5 [SPARK-16060][SQL] Support Vectorized ORC Reader
## What changes were proposed in this pull request?

This PR adds an ORC columnar-batch reader to native `OrcFileFormat`. Since both Spark `ColumnarBatch` and ORC `RowBatch` are used together, it is faster than the current Spark implementation. This replaces the prior PR, #17924.

Also, this PR adds `OrcReadBenchmark` to show the performance improvement.

## How was this patch tested?

Pass the existing test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19943 from dongjoon-hyun/SPARK-16060.
2018-01-09 21:48:14 +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
gatorsmile 9a7048b288 [HOTFIX] Fix style checking failure
## What changes were proposed in this pull request?
This PR is to fix the  style checking failure.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20175 from gatorsmile/stylefix.
2018-01-07 00:19:21 +08:00
fjh100456 7b78041423 [SPARK-21786][SQL] When acquiring 'compressionCodecClassName' in 'ParquetOptions', parquet.compression needs to be considered.
[SPARK-21786][SQL] When acquiring 'compressionCodecClassName' in 'ParquetOptions', `parquet.compression` needs to be considered.

## What changes were proposed in this pull request?
Since Hive 1.1, Hive allows users to set parquet compression codec via table-level properties parquet.compression. See the JIRA: https://issues.apache.org/jira/browse/HIVE-7858 . We do support orc.compression for ORC. Thus, for external users, it is more straightforward to support both. See the stackflow question: https://stackoverflow.com/questions/36941122/spark-sql-ignores-parquet-compression-propertie-specified-in-tblproperties
In Spark side, our table-level compression conf compression was added by #11464 since Spark 2.0.
We need to support both table-level conf. Users might also use session-level conf spark.sql.parquet.compression.codec. The priority rule will be like
If other compression codec configuration was found through hive or parquet, the precedence would be compression, parquet.compression, spark.sql.parquet.compression.codec. Acceptable values include: none, uncompressed, snappy, gzip, lzo.
The rule for Parquet is consistent with the ORC after the change.

Changes:
1.Increased acquiring 'compressionCodecClassName' from `parquet.compression`,and the precedence order is `compression`,`parquet.compression`,`spark.sql.parquet.compression.codec`, just like what we do in `OrcOptions`.

2.Change `spark.sql.parquet.compression.codec` to support "none".Actually in `ParquetOptions`,we do support "none" as equivalent to "uncompressed", but it does not allowed to configured to "none".

3.Change `compressionCode` to `compressionCodecClassName`.

## How was this patch tested?
Add test.

Author: fjh100456 <fu.jinhua6@zte.com.cn>

Closes #20076 from fjh100456/ParquetOptionIssue.
2018-01-06 18:19:57 +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
Juliusz Sompolski df7fc3ef38 [SPARK-22957] ApproxQuantile breaks if the number of rows exceeds MaxInt
## What changes were proposed in this pull request?

32bit Int was used for row rank.
That overflowed in a dataframe with more than 2B rows.

## How was this patch tested?

Added test, but ignored, as it takes 4 minutes.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #20152 from juliuszsompolski/SPARK-22957.
2018-01-05 10:16:34 +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
Wenchen Fan a66fe36cee [SPARK-20236][SQL] dynamic partition overwrite
## What changes were proposed in this pull request?

When overwriting a partitioned table with dynamic partition columns, the behavior is different between data source and hive tables.

data source table: delete all partition directories that match the static partition values provided in the insert statement.

hive table: only delete partition directories which have data written into it

This PR adds a new config to make users be able to choose hive's behavior.

## How was this patch tested?

new tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18714 from cloud-fan/overwrite-partition.
2018-01-03 22:18:13 +08:00
gatorsmile 1a87a1609c [SPARK-22934][SQL] Make optional clauses order insensitive for CREATE TABLE SQL statement
## What changes were proposed in this pull request?
Currently, our CREATE TABLE syntax require the EXACT order of clauses. It is pretty hard to remember the exact order. Thus, this PR is to make optional clauses order insensitive for `CREATE TABLE` SQL statement.

```
CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db_name.]table_name
    [(col_name1 col_type1 [COMMENT col_comment1], ...)]
    USING datasource
    [OPTIONS (key1=val1, key2=val2, ...)]
    [PARTITIONED BY (col_name1, col_name2, ...)]
    [CLUSTERED BY (col_name3, col_name4, ...) INTO num_buckets BUCKETS]
    [LOCATION path]
    [COMMENT table_comment]
    [TBLPROPERTIES (key1=val1, key2=val2, ...)]
    [AS select_statement]
```

The proposal is to make the following clauses order insensitive.
```
    [OPTIONS (key1=val1, key2=val2, ...)]
    [PARTITIONED BY (col_name1, col_name2, ...)]
    [CLUSTERED BY (col_name3, col_name4, ...) INTO num_buckets BUCKETS]
    [LOCATION path]
    [COMMENT table_comment]
    [TBLPROPERTIES (key1=val1, key2=val2, ...)]
```

The same idea is also applicable to Create Hive Table.
```
CREATE [EXTERNAL] TABLE [IF NOT EXISTS] [db_name.]table_name
    [(col_name1[:] col_type1 [COMMENT col_comment1], ...)]
    [COMMENT table_comment]
    [PARTITIONED BY (col_name2[:] col_type2 [COMMENT col_comment2], ...)]
    [ROW FORMAT row_format]
    [STORED AS file_format]
    [LOCATION path]
    [TBLPROPERTIES (key1=val1, key2=val2, ...)]
    [AS select_statement]
```

The proposal is to make the following clauses order insensitive.
```
    [COMMENT table_comment]
    [PARTITIONED BY (col_name2[:] col_type2 [COMMENT col_comment2], ...)]
    [ROW FORMAT row_format]
    [STORED AS file_format]
    [LOCATION path]
    [TBLPROPERTIES (key1=val1, key2=val2, ...)]
```

## How was this patch tested?
Added test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20133 from gatorsmile/createDataSourceTableDDL.
2018-01-03 22:09:30 +08:00
Juliusz Sompolski 247a08939d [SPARK-22938] Assert that SQLConf.get is accessed only on the driver.
## What changes were proposed in this pull request?

Assert if code tries to access SQLConf.get on executor.
This can lead to hard to detect bugs, where the executor will read fallbackConf, falling back to default config values, ignoring potentially changed non-default configs.
If a config is to be passed to executor code, it needs to be read on the driver, and passed explicitly.

## How was this patch tested?

Check in existing tests.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #20136 from juliuszsompolski/SPARK-22938.
2018-01-03 21:40:51 +08:00
gatorsmile e0c090f227 [SPARK-22932][SQL] Refactor AnalysisContext
## What changes were proposed in this pull request?
Add a `reset` function to ensure the state in `AnalysisContext ` is per-query.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20127 from gatorsmile/refactorAnalysisContext.
2018-01-02 09:19:18 +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
Zhenhua Wang 224375c55f [SPARK-22892][SQL] Simplify some estimation logic by using double instead of decimal
## What changes were proposed in this pull request?

Simplify some estimation logic by using double instead of decimal.

## How was this patch tested?

Existing tests.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #20062 from wzhfy/simplify_by_double.
2017-12-29 15:39:56 +08:00
Wenchen Fan 755f2f5189 [SPARK-20392][SQL][FOLLOWUP] should not add extra AnalysisBarrier
## What changes were proposed in this pull request?

I found this problem while auditing the analyzer code. It's dangerous to introduce extra `AnalysisBarrer` during analysis, as the plan inside it will bypass all analysis afterward, which may not be expected. We should only preserve `AnalysisBarrer` but not introduce new ones.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20094 from cloud-fan/barrier.
2017-12-28 21:33:03 +08:00
Kazuaki Ishizaki 5683984520 [SPARK-18016][SQL][FOLLOW-UP] Code Generation: Constant Pool Limit - reduce entries for mutable state
## What changes were proposed in this pull request?

This PR addresses additional review comments in #19811

## How was this patch tested?

Existing test suites

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

Closes #20036 from kiszk/SPARK-18066-followup.
2017-12-28 12:28:19 +08:00
Yuming Wang 91d1b300d4 [SPARK-22894][SQL] DateTimeOperations should accept SQL like string type
## What changes were proposed in this pull request?

`DateTimeOperations` accept [`StringType`](ae998ec2b5/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala (L669)),  but:

```
spark-sql> SELECT '2017-12-24' + interval 2 months 2 seconds;
Error in query: cannot resolve '(CAST('2017-12-24' AS DOUBLE) + interval 2 months 2 seconds)' due to data type mismatch: differing types in '(CAST('2017-12-24' AS DOUBLE) + interval 2 months 2 seconds)' (double and calendarinterval).; line 1 pos 7;
'Project [unresolvedalias((cast(2017-12-24 as double) + interval 2 months 2 seconds), None)]
+- OneRowRelation
spark-sql>
```

After this PR:
```
spark-sql> SELECT '2017-12-24' + interval 2 months 2 seconds;
2018-02-24 00:00:02
Time taken: 0.2 seconds, Fetched 1 row(s)

```

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #20067 from wangyum/SPARK-22894.
2017-12-26 09:40:41 -08:00
Wenchen Fan 9348e68420 [SPARK-22833][EXAMPLE] Improvement SparkHive Scala Examples
## What changes were proposed in this pull request?
Some improvements:
1. Point out we are using both Spark SQ native syntax and HQL syntax in the example
2. Avoid using the same table name with temp view, to not confuse users.
3. Create the external hive table with a directory that already has data, which is a more common use case.
4. Remove the usage of `spark.sql.parquet.writeLegacyFormat`. This config was introduced by https://github.com/apache/spark/pull/8566 and has nothing to do with Hive.
5. Remove `repartition` and `coalesce` example. These 2 are not Hive specific, we should put them in a different example file. BTW they can't accurately control the number of output files, `spark.sql.files.maxRecordsPerFile` also controls it.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20081 from cloud-fan/minor.
2017-12-26 09:37:39 -08:00
Yuming Wang 33ae2437ba [SPARK-22893][SQL] Unified the data type mismatch message
## What changes were proposed in this pull request?

We should use `dataType.simpleString` to unified the data type mismatch message:
Before:
```
spark-sql> select cast(1 as binary);
Error in query: cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast IntegerType to BinaryType; line 1 pos 7;
```
After:
```
park-sql> select cast(1 as binary);
Error in query: cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast int to binary; line 1 pos 7;
```

## How was this patch tested?

Exist test.

Author: Yuming Wang <wgyumg@gmail.com>

Closes #20064 from wangyum/SPARK-22893.
2017-12-25 01:14:09 -08:00
Jose Torres 8941a4abca [SPARK-22789] Map-only continuous processing execution
## What changes were proposed in this pull request?

Basic continuous execution, supporting map/flatMap/filter, with commits and advancement through RPC.

## How was this patch tested?

new unit-ish tests (exercising execution end to end)

Author: Jose Torres <jose@databricks.com>

Closes #19984 from jose-torres/continuous-impl.
2017-12-22 23:05:03 -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
Yuming Wang 4e107fdb74 [SPARK-22822][TEST] Basic tests for WindowFrameCoercion and DecimalPrecision
## What changes were proposed in this pull request?

Test Coverage for `WindowFrameCoercion` and `DecimalPrecision`, this is a Sub-tasks for [SPARK-22722](https://issues.apache.org/jira/browse/SPARK-22722).

## How was this patch tested?

N/A

Author: Yuming Wang <wgyumg@gmail.com>

Closes #20008 from wangyum/SPARK-22822.
2017-12-21 09:18:27 -08:00
Wenchen Fan 8a0ed5a5ee [SPARK-22668][SQL] Ensure no global variables in arguments of method split by CodegenContext.splitExpressions()
## What changes were proposed in this pull request?

Passing global variables to the split method is dangerous, as any mutating to it is ignored and may lead to unexpected behavior.

To prevent this, one approach is to make sure no expression would output global variables: Localizing lifetime of mutable states in expressions.

Another approach is, when calling `ctx.splitExpression`, make sure we don't use children's output as parameter names.

Approach 1 is actually hard to do, as we need to check all expressions and operators that support whole-stage codegen. Approach 2 is easier as the callers of `ctx.splitExpressions` are not too many.

Besides, approach 2 is more flexible, as children's output may be other stuff that can't be parameter name: literal, inlined statement(a + 1), etc.

close https://github.com/apache/spark/pull/19865
close https://github.com/apache/spark/pull/19938

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20021 from cloud-fan/codegen.
2017-12-22 00:21:27 +08:00
Kazuaki Ishizaki cb9fc8d9b6 [SPARK-22848][SQL] Eliminate mutable state from Stack
## What changes were proposed in this pull request?

This PR eliminates mutable states from the generated code for `Stack`.

## How was this patch tested?

Existing test suites

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

Closes #20035 from kiszk/SPARK-22848.
2017-12-21 14:54:38 +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
gatorsmile ef10f452e6 [SPARK-21652][SQL][FOLLOW-UP] Fix rule conflict caused by InferFiltersFromConstraints
## What changes were proposed in this pull request?
The optimizer rule `InferFiltersFromConstraints` could trigger our batch  `Operator Optimizations` exceeds the max iteration limit (i.e., 100) so that the final plan might not be properly optimized. The rule `InferFiltersFromConstraints` could conflict with the other Filter/Join predicate reduction rules. Thus, we need to separate `InferFiltersFromConstraints` from the other rules.

This PR is to separate `InferFiltersFromConstraints ` from the main batch `Operator Optimizations` .

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19149 from gatorsmile/inferFilterRule.
2017-12-19 09:05:47 -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
gatorsmile b779c93518 [SPARK-22815][SQL] Keep PromotePrecision in Optimized Plans
## What changes were proposed in this pull request?
We could get incorrect results by running DecimalPrecision twice. This PR resolves the original found in https://github.com/apache/spark/pull/15048 and https://github.com/apache/spark/pull/14797. After this PR, it becomes easier to change it back using `children` instead of using `innerChildren`.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20000 from gatorsmile/keepPromotePrecision.
2017-12-19 22:17:04 +08:00
gatorsmile 28315714dd [SPARK-22791][SQL][SS] Redact Output of Explain
## What changes were proposed in this pull request?

When calling explain on a query, the output can contain sensitive information. We should provide an admin/user to redact such information.

Before this PR, the plan of SS is like this
```
== Physical Plan ==
*HashAggregate(keys=[value#6], functions=[count(1)], output=[value#6, count(1)#12L])
+- StateStoreSave [value#6], state info [ checkpoint = file:/private/var/folders/vx/j0ydl5rn0gd9mgrh1pljnw900000gn/T/temporary-91c6fac0-609f-4bc8-ad57-52c189f06797/state, runId = 05a4b3af-f02c-40f8-9ff9-a3e18bae496f, opId = 0, ver = 0, numPartitions = 5], Complete, 0
   +- *HashAggregate(keys=[value#6], functions=[merge_count(1)], output=[value#6, count#18L])
      +- StateStoreRestore [value#6], state info [ checkpoint = file:/private/var/folders/vx/j0ydl5rn0gd9mgrh1pljnw900000gn/T/temporary-91c6fac0-609f-4bc8-ad57-52c189f06797/state, runId = 05a4b3af-f02c-40f8-9ff9-a3e18bae496f, opId = 0, ver = 0, numPartitions = 5]
         +- *HashAggregate(keys=[value#6], functions=[merge_count(1)], output=[value#6, count#18L])
            +- Exchange hashpartitioning(value#6, 5)
               +- *HashAggregate(keys=[value#6], functions=[partial_count(1)], output=[value#6, count#18L])
                  +- *SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#6]
                     +- *MapElements <function1>, obj#5: java.lang.String
                        +- *DeserializeToObject value#30.toString, obj#4: java.lang.String
                           +- LocalTableScan [value#30]
```

After this PR, we can get the following output if users set `spark.redaction.string.regex` to `file:/[\\w_]+`
```
== Physical Plan ==
*HashAggregate(keys=[value#6], functions=[count(1)], output=[value#6, count(1)#12L])
+- StateStoreSave [value#6], state info [ checkpoint = *********(redacted)/var/folders/vx/j0ydl5rn0gd9mgrh1pljnw900000gn/T/temporary-e7da9b7d-3ec0-474d-8b8c-927f7d12ed72/state, runId = 8a9c3761-93d5-4896-ab82-14c06240dcea, opId = 0, ver = 0, numPartitions = 5], Complete, 0
   +- *HashAggregate(keys=[value#6], functions=[merge_count(1)], output=[value#6, count#32L])
      +- StateStoreRestore [value#6], state info [ checkpoint = *********(redacted)/var/folders/vx/j0ydl5rn0gd9mgrh1pljnw900000gn/T/temporary-e7da9b7d-3ec0-474d-8b8c-927f7d12ed72/state, runId = 8a9c3761-93d5-4896-ab82-14c06240dcea, opId = 0, ver = 0, numPartitions = 5]
         +- *HashAggregate(keys=[value#6], functions=[merge_count(1)], output=[value#6, count#32L])
            +- Exchange hashpartitioning(value#6, 5)
               +- *HashAggregate(keys=[value#6], functions=[partial_count(1)], output=[value#6, count#32L])
                  +- *SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#6]
                     +- *MapElements <function1>, obj#5: java.lang.String
                        +- *DeserializeToObject value#27.toString, obj#4: java.lang.String
                           +- LocalTableScan [value#27]
```
## How was this patch tested?
Added a test case

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19985 from gatorsmile/redactPlan.
2017-12-19 22:12:23 +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
gatorsmile d4e69595dd [MINOR][SQL] Remove Useless zipWithIndex from ResolveAliases
## What changes were proposed in this pull request?
Remove useless `zipWithIndex` from `ResolveAliases `.

## How was this patch tested?
The existing tests

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20009 from gatorsmile/try22.
2017-12-19 09:48:31 +08:00
Marcelo Vanzin 772e4648d9 [SPARK-20653][CORE] Add cleaning of old elements from the status store.
This change restores the functionality that keeps a limited number of
different types (jobs, stages, etc) depending on configuration, to avoid
the store growing indefinitely over time.

The feature is implemented by creating a new type (ElementTrackingStore)
that wraps a KVStore and allows triggers to be set up for when elements
of a certain type meet a certain threshold. Triggers don't need to
necessarily only delete elements, but the current API is set up in a way
that makes that use case easier.

The new store also has a trigger for the "close" call, which makes it
easier for listeners to register code for cleaning things up and flushing
partial state to the store.

The old configurations for cleaning up the stored elements from the core
and SQL UIs are now active again, and the old unit tests are re-enabled.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19751 from vanzin/SPARK-20653.
2017-12-18 14:08:48 -06:00
Marcelo Vanzin c3dd2a26de [SPARK-22779][SQL] Resolve default values for fallback configs.
SQLConf allows some callers to define a custom default value for
configs, and that complicates a little bit the handling of fallback
config entries, since most of the default value resolution is
hidden by the config code.

This change peaks into the internals of these fallback configs
to figure out the correct default value, and also returns the
current human-readable default when showing the default value
(e.g. through "set -v").

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19974 from vanzin/SPARK-22779.
2017-12-13 22:46:20 -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
Wenchen Fan bc7e4a90c0 Revert "[SPARK-22600][SQL][FOLLOW-UP] Fix a compilation error in TPCDS q75/q77"
This reverts commit ef92999653.
2017-12-14 11:21:34 +08:00
Takeshi Yamamuro ef92999653 [SPARK-22600][SQL][FOLLOW-UP] Fix a compilation error in TPCDS q75/q77
## What changes were proposed in this pull request?
This pr fixed a compilation error of TPCDS `q75`/`q77`  caused by #19813;
```
  java.util.concurrent.ExecutionException: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 371, Column 16: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 371, Column 16: Expression "bhj_matched" is not an rvalue
  at com.google.common.util.concurrent.AbstractFuture$Sync.getValue(AbstractFuture.java:306)
  at com.google.common.util.concurrent.AbstractFuture$Sync.get(AbstractFuture.java:293)
  at com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:116)
  at com.google.common.util.concurrent.Uninterruptibles.getUninterruptibly(Uninterruptibles.java:135)
```

## How was this patch tested?
Manually checked `q75`/`q77` can be properly compiled

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #19969 from maropu/SPARK-22600-FOLLOWUP.
2017-12-13 15:55:16 -08:00
Liang-Chi Hsieh ba0e79f57c [SPARK-22772][SQL] Use splitExpressionsWithCurrentInputs to split codes in elt
## What changes were proposed in this pull request?

In SPARK-22550 which fixes 64KB JVM bytecode limit problem with elt, `buildCodeBlocks` is used to split codes. However, we should use `splitExpressionsWithCurrentInputs` because it considers both normal and wholestage codgen (it is not supported yet, so it simply doesn't split the codes).

## How was this patch tested?

Existing tests.

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

Closes #19964 from viirya/SPARK-22772.
2017-12-13 13:54:16 -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
Wenchen Fan bdb5e55c2a [SPARK-21322][SQL][FOLLOWUP] support histogram in filter cardinality estimation
## What changes were proposed in this pull request?

some code cleanup/refactor and naming improvement.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19952 from cloud-fan/minor.
2017-12-13 14:49:15 +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
Imran Rashid acf7ef3154 [SPARK-12297][SQL] Adjust timezone for int96 data from impala
## What changes were proposed in this pull request?

Int96 data written by impala vs data written by hive & spark is stored slightly differently -- they use a different offset for the timezone.  This adds an option "spark.sql.parquet.int96TimestampConversion" (false by default) to adjust timestamps if and only if the writer is impala (or more precisely, if the parquet file's "createdBy" metadata does not start with "parquet-mr").  This matches the existing behavior in hive from HIVE-9482.

## How was this patch tested?

Unit test added, existing tests run via jenkins.

Author: Imran Rashid <irashid@cloudera.com>
Author: Henry Robinson <henry@apache.org>

Closes #19769 from squito/SPARK-12297_skip_conversion.
2017-12-09 11:53:15 +09:00
Wang Gengliang 18b75d465b [SPARK-22719][SQL] Refactor ConstantPropagation
## What changes were proposed in this pull request?

The current time complexity of ConstantPropagation is O(n^2), which can be slow when the query is complex.
Refactor the implementation with O( n ) time complexity, and some pruning to avoid traversing the whole `Condition`

## How was this patch tested?

Unit test.

Also simple benchmark test in ConstantPropagationSuite
```
  val condition = (1 to 500).map{_ => Rand(0) === Rand(0)}.reduce(And)
  val query = testRelation
    .select(columnA)
    .where(condition)
  val start = System.currentTimeMillis()
  (1 to 40).foreach { _ =>
    Optimize.execute(query.analyze)
  }
  val end = System.currentTimeMillis()
  println(end - start)
```
Run time before changes: 18989ms (474ms per loop)
Run time after changes: 1275 ms (32ms per loop)

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #19912 from gengliangwang/ConstantPropagation.
2017-12-07 10:24: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
Kazuaki Ishizaki 8ae004b460 [SPARK-22688][SQL] Upgrade Janino version to 3.0.8
## What changes were proposed in this pull request?

This PR upgrade Janino version to 3.0.8. [Janino 3.0.8](https://janino-compiler.github.io/janino/changelog.html) includes an important fix to reduce the number of constant pool entries by using 'sipush' java bytecode.

* SIPUSH bytecode is not used for short integer constant [#33](https://github.com/janino-compiler/janino/issues/33).

Please see detail in [this discussion thread](https://github.com/apache/spark/pull/19518#issuecomment-346674976).

## How was this patch tested?

Existing tests

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

Closes #19890 from kiszk/SPARK-22688.
2017-12-06 16:15:25 -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
gatorsmile effca9868e [SPARK-22720][SS] Make EventTimeWatermark Extend UnaryNode
## What changes were proposed in this pull request?
Our Analyzer and Optimizer have multiple rules for `UnaryNode`. After making `EventTimeWatermark` extend `UnaryNode`, we do not need a special handling for `EventTimeWatermark`.

## How was this patch tested?
The existing tests

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19913 from gatorsmile/eventtimewatermark.
2017-12-06 13:11:38 -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 132a3f4708 [SPARK-22500][SQL][FOLLOWUP] cast for struct can split code even with whole stage codegen
## What changes were proposed in this pull request?

A followup of https://github.com/apache/spark/pull/19730, we can split the code for casting struct even with whole stage codegen.

This PR also has some renaming to make the code easier to read.

## How was this patch tested?

existing test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19891 from cloud-fan/cast.
2017-12-05 11:40:13 -08:00
Wenchen Fan ced6ccf0d6 [SPARK-22701][SQL] add ctx.splitExpressionsWithCurrentInputs
## What changes were proposed in this pull request?

This pattern appears many times in the codebase:
```
if (ctx.INPUT_ROW == null || ctx.currentVars != null) {
  exprs.mkString("\n")
} else {
  ctx.splitExpressions(...)
}
```

This PR adds a `ctx.splitExpressionsWithCurrentInputs` for this pattern

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19895 from cloud-fan/splitExpression.
2017-12-05 10:15:15 -08:00
Dongjoon Hyun 326f1d6728 [SPARK-20728][SQL] Make OrcFileFormat configurable between sql/hive and sql/core
## What changes were proposed in this pull request?

This PR aims to provide a configuration to choose the default `OrcFileFormat` from legacy `sql/hive` module or new `sql/core` module.

For example, this configuration will affects the following operations.
```scala
spark.read.orc(...)
```
```sql
CREATE TABLE t
USING ORC
...
```

## How was this patch tested?

Pass the Jenkins with new test suites.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19871 from dongjoon-hyun/spark-sql-orc-enabled.
2017-12-05 20:46:35 +08:00
gatorsmile 53e5251bb3 [SPARK-22675][SQL] Refactoring PropagateTypes in TypeCoercion
## What changes were proposed in this pull request?
PropagateTypes are called twice in TypeCoercion. We do not need to call it twice. Instead, we should call it after each change on the types.

## How was this patch tested?
The existing tests

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19874 from gatorsmile/deduplicatePropagateTypes.
2017-12-05 20:43:02 +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
Marco Gaido 2c16267f7c [SPARK-22669][SQL] Avoid unnecessary function calls in code generation
## What changes were proposed in this pull request?

In many parts of the codebase for code generation, we are splitting the code to avoid exceptions due to the 64KB method size limit. This is generating a lot of methods which are called every time, even though sometime this is not needed. As pointed out here: https://github.com/apache/spark/pull/19752#discussion_r153081547, this is a not negligible overhead which can be avoided.

The PR applies the same approach used in #19752 also to the other places where this was feasible.

## How was this patch tested?

existing UTs.

Author: Marco Gaido <mgaido@hortonworks.com>

Closes #19860 from mgaido91/SPARK-22669.
2017-12-03 22:56:03 +08: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
Kazuaki Ishizaki 284836862b [SPARK-22608][SQL] add new API to CodeGeneration.splitExpressions()
## What changes were proposed in this pull request?

This PR adds a new API to ` CodeGenenerator.splitExpression` since since several ` CodeGenenerator.splitExpression` are used with `ctx.INPUT_ROW` to avoid code duplication.

## How was this patch tested?

Used existing test suits

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

Closes #19821 from kiszk/SPARK-22608.
2017-11-30 01:19:37 +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
Zhenhua Wang da35574297 [SPARK-22515][SQL] Estimation relation size based on numRows * rowSize
## What changes were proposed in this pull request?

Currently, relation size is computed as the sum of file size, which is error-prone because storage format like parquet may have a much smaller file size compared to in-memory size. When we choose broadcast join based on file size, there's a risk of OOM. But if the number of rows is available in statistics, we can get a better estimation by `numRows * rowSize`, which helps to alleviate this problem.

## How was this patch tested?

Added a new test case for data source table and hive table.

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

Closes #19743 from wzhfy/better_leaf_size.
2017-11-28 11:43:21 -08:00
Wenchen Fan b70e483cb3 [SPARK-22617][SQL] make splitExpressions extract current input of the context
## What changes were proposed in this pull request?

Mostly when we call `CodegenContext.splitExpressions`, we want to split the code into methods and pass the current inputs of the codegen context to these methods so that the code in these methods can still be evaluated.

This PR makes the expectation clear, while still keep the advanced version of `splitExpressions` to customize the inputs to pass to generated methods.

## How was this patch tested?

existing test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19827 from cloud-fan/codegen.
2017-11-28 22:57:30 +08:00
Wenchen Fan 1e07fff248 [SPARK-22520][SQL][FOLLOWUP] remove outer if for case when codegen
## What changes were proposed in this pull request?

a minor cleanup for https://github.com/apache/spark/pull/19752 . Remove the outer if as the code is inside `do while`

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19830 from cloud-fan/minor.
2017-11-28 22:43:24 +08:00
Takuya UESHIN 64817c423c [SPARK-22395][SQL][PYTHON] Fix the behavior of timestamp values for Pandas to respect session timezone
## What changes were proposed in this pull request?

When converting Pandas DataFrame/Series from/to Spark DataFrame using `toPandas()` or pandas udfs, timestamp values behave to respect Python system timezone instead of session timezone.

For example, let's say we use `"America/Los_Angeles"` as session timezone and have a timestamp value `"1970-01-01 00:00:01"` in the timezone. Btw, I'm in Japan so Python timezone would be `"Asia/Tokyo"`.

The timestamp value from current `toPandas()` will be the following:

```
>>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles")
>>> df = spark.createDataFrame([28801], "long").selectExpr("timestamp(value) as ts")
>>> df.show()
+-------------------+
|                 ts|
+-------------------+
|1970-01-01 00:00:01|
+-------------------+

>>> df.toPandas()
                   ts
0 1970-01-01 17:00:01
```

As you can see, the value becomes `"1970-01-01 17:00:01"` because it respects Python timezone.
As we discussed in #18664, we consider this behavior is a bug and the value should be `"1970-01-01 00:00:01"`.

## How was this patch tested?

Added tests and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #19607 from ueshin/issues/SPARK-22395.
2017-11-28 16:45:22 +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
Zhenhua Wang 1ff4a77be4 [SPARK-22529][SQL] Relation stats should be consistent with other plans based on cbo config
## What changes were proposed in this pull request?

Currently, relation stats is the same whether cbo is enabled or not. While relation (`LogicalRelation` or `HiveTableRelation`) is a `LogicalPlan`, its behavior is inconsistent with other plans. This can cause confusion when user runs EXPLAIN COST commands. Besides, when CBO is disabled, we apply the size-only estimation strategy, so there's no need to propagate other catalog statistics to relation.

## How was this patch tested?

Enhanced existing tests case and added a test case.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #19757 from wzhfy/catalog_stats_conversion.
2017-11-28 01:13:44 +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
Sean Owen fba63c1a7b [SPARK-22607][BUILD] Set large stack size consistently for tests to avoid StackOverflowError
## What changes were proposed in this pull request?

Set `-ea` and `-Xss4m` consistently for tests, to fix in particular:

```
OrderingSuite:
...
- GenerateOrdering with ShortType
*** RUN ABORTED ***
java.lang.StackOverflowError:
at org.codehaus.janino.CodeContext.flowAnalysis(CodeContext.java:370)
at org.codehaus.janino.CodeContext.flowAnalysis(CodeContext.java:541)
at org.codehaus.janino.CodeContext.flowAnalysis(CodeContext.java:541)
at org.codehaus.janino.CodeContext.flowAnalysis(CodeContext.java:541)
at org.codehaus.janino.CodeContext.flowAnalysis(CodeContext.java:541)
at org.codehaus.janino.CodeContext.flowAnalysis(CodeContext.java:541)
at org.codehaus.janino.CodeContext.flowAnalysis(CodeContext.java:541)
at org.codehaus.janino.CodeContext.flowAnalysis(CodeContext.java:541)
...
```

## How was this patch tested?

Existing tests. Manually verified it resolves the StackOverflowError this intends to resolve.

Author: Sean Owen <sowen@cloudera.com>

Closes #19820 from srowen/SPARK-22607.
2017-11-26 07:42:44 -06:00
Wenchen Fan 70221903f5 [SPARK-22596][SQL] set ctx.currentVars in CodegenSupport.consume
## What changes were proposed in this pull request?

`ctx.currentVars` means the input variables for the current operator, which is already decided in `CodegenSupport`, we can set it there instead of `doConsume`.

also add more comments to help people understand the codegen framework.

After this PR, we now have a principle about setting `ctx.currentVars` and `ctx.INPUT_ROW`:
1. for non-whole-stage-codegen path, never set them. (permit some special cases like generating ordering)
2. for whole-stage-codegen `produce` path, mostly we don't need to set them, but blocking operators may need to set them for expressions that produce data from data source, sort buffer, aggregate buffer, etc.
3. for whole-stage-codegen `consume` path, mostly we don't need to set them because `currentVars` is automatically set to child input variables and `INPUT_ROW` is mostly not used. A few plans need to tweak them as they may have different inputs, or they use the input row.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19803 from cloud-fan/codegen.
2017-11-24 21:50:30 -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 572af5027e [SPARK-20101][SQL][FOLLOW-UP] use correct config name "spark.sql.columnVector.offheap.enabled"
## What changes were proposed in this pull request?

This PR addresses [the spelling miss](https://github.com/apache/spark/pull/17436#discussion_r152189670) of the config name `spark.sql.columnVector.offheap.enabled`.
We should use `spark.sql.columnVector.offheap.enabled`.

## How was this patch tested?

Existing tests

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

Closes #19794 from kiszk/SPARK-20101-follow.
2017-11-22 13:27:20 +01: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
hyukjinkwon 6d7ebf2f9f [SPARK-22165][SQL] Fixes type conflicts between double, long, decimals, dates and timestamps in partition column
## What changes were proposed in this pull request?

This PR proposes to add a rule that re-uses `TypeCoercion.findWiderCommonType` when resolving type conflicts in partition values.

Currently, this uses numeric precedence-like comparison; therefore, it looks introducing failures for type conflicts between timestamps, dates and decimals, please see:

```scala
private val upCastingOrder: Seq[DataType] =
  Seq(NullType, IntegerType, LongType, FloatType, DoubleType, StringType)
...
literals.map(_.dataType).maxBy(upCastingOrder.indexOf(_))
```

The codes below:

```scala
val df = Seq((1, "2015-01-01"), (2, "2016-01-01 00:00:00")).toDF("i", "ts")
df.write.format("parquet").partitionBy("ts").save("/tmp/foo")
spark.read.load("/tmp/foo").printSchema()

val df = Seq((1, "1"), (2, "1" * 30)).toDF("i", "decimal")
df.write.format("parquet").partitionBy("decimal").save("/tmp/bar")
spark.read.load("/tmp/bar").printSchema()
```

produces output as below:

**Before**

```
root
 |-- i: integer (nullable = true)
 |-- ts: date (nullable = true)

root
 |-- i: integer (nullable = true)
 |-- decimal: integer (nullable = true)
```

**After**

```
root
 |-- i: integer (nullable = true)
 |-- ts: timestamp (nullable = true)

root
 |-- i: integer (nullable = true)
 |-- decimal: decimal(30,0) (nullable = true)
```

### Type coercion table:

This PR proposes the type conflict resolusion as below:

**Before**

|InputA \ InputB|`NullType`|`IntegerType`|`LongType`|`DecimalType(38,0)`|`DoubleType`|`DateType`|`TimestampType`|`StringType`|
|------------------------|----------|----------|----------|----------|----------|----------|----------|----------|
|**`NullType`**|`StringType`|`IntegerType`|`LongType`|`StringType`|`DoubleType`|`StringType`|`StringType`|`StringType`|
|**`IntegerType`**|`IntegerType`|`IntegerType`|`LongType`|`IntegerType`|`DoubleType`|`IntegerType`|`IntegerType`|`StringType`|
|**`LongType`**|`LongType`|`LongType`|`LongType`|`LongType`|`DoubleType`|`LongType`|`LongType`|`StringType`|
|**`DecimalType(38,0)`**|`StringType`|`IntegerType`|`LongType`|`DecimalType(38,0)`|`DoubleType`|`DecimalType(38,0)`|`DecimalType(38,0)`|`StringType`|
|**`DoubleType`**|`DoubleType`|`DoubleType`|`DoubleType`|`DoubleType`|`DoubleType`|`DoubleType`|`DoubleType`|`StringType`|
|**`DateType`**|`StringType`|`IntegerType`|`LongType`|`DateType`|`DoubleType`|`DateType`|`DateType`|`StringType`|
|**`TimestampType`**|`StringType`|`IntegerType`|`LongType`|`TimestampType`|`DoubleType`|`TimestampType`|`TimestampType`|`StringType`|
|**`StringType`**|`StringType`|`StringType`|`StringType`|`StringType`|`StringType`|`StringType`|`StringType`|`StringType`|

**After**

|InputA \ InputB|`NullType`|`IntegerType`|`LongType`|`DecimalType(38,0)`|`DoubleType`|`DateType`|`TimestampType`|`StringType`|
|------------------------|----------|----------|----------|----------|----------|----------|----------|----------|
|**`NullType`**|`NullType`|`IntegerType`|`LongType`|`DecimalType(38,0)`|`DoubleType`|`DateType`|`TimestampType`|`StringType`|
|**`IntegerType`**|`IntegerType`|`IntegerType`|`LongType`|`DecimalType(38,0)`|`DoubleType`|`StringType`|`StringType`|`StringType`|
|**`LongType`**|`LongType`|`LongType`|`LongType`|`DecimalType(38,0)`|`StringType`|`StringType`|`StringType`|`StringType`|
|**`DecimalType(38,0)`**|`DecimalType(38,0)`|`DecimalType(38,0)`|`DecimalType(38,0)`|`DecimalType(38,0)`|`StringType`|`StringType`|`StringType`|`StringType`|
|**`DoubleType`**|`DoubleType`|`DoubleType`|`StringType`|`StringType`|`DoubleType`|`StringType`|`StringType`|`StringType`|
|**`DateType`**|`DateType`|`StringType`|`StringType`|`StringType`|`StringType`|`DateType`|`TimestampType`|`StringType`|
|**`TimestampType`**|`TimestampType`|`StringType`|`StringType`|`StringType`|`StringType`|`TimestampType`|`TimestampType`|`StringType`|
|**`StringType`**|`StringType`|`StringType`|`StringType`|`StringType`|`StringType`|`StringType`|`StringType`|`StringType`|

This was produced by:

```scala
  test("Print out chart") {
    val supportedTypes: Seq[DataType] = Seq(
      NullType, IntegerType, LongType, DecimalType(38, 0), DoubleType,
      DateType, TimestampType, StringType)

    // Old type conflict resolution:
    val upCastingOrder: Seq[DataType] =
      Seq(NullType, IntegerType, LongType, FloatType, DoubleType, StringType)
    def oldResolveTypeConflicts(dataTypes: Seq[DataType]): DataType = {
      val topType = dataTypes.maxBy(upCastingOrder.indexOf(_))
      if (topType == NullType) StringType else topType
    }
    println(s"|InputA \\ InputB|${supportedTypes.map(dt => s"`${dt.toString}`").mkString("|")}|")
    println(s"|------------------------|${supportedTypes.map(_ => "----------").mkString("|")}|")
    supportedTypes.foreach { inputA =>
      val types = supportedTypes.map(inputB => oldResolveTypeConflicts(Seq(inputA, inputB)))
      println(s"|**`$inputA`**|${types.map(dt => s"`${dt.toString}`").mkString("|")}|")
    }

    // New type conflict resolution:
    def newResolveTypeConflicts(dataTypes: Seq[DataType]): DataType = {
      dataTypes.fold[DataType](NullType)(findWiderTypeForPartitionColumn)
    }
    println(s"|InputA \\ InputB|${supportedTypes.map(dt => s"`${dt.toString}`").mkString("|")}|")
    println(s"|------------------------|${supportedTypes.map(_ => "----------").mkString("|")}|")
    supportedTypes.foreach { inputA =>
      val types = supportedTypes.map(inputB => newResolveTypeConflicts(Seq(inputA, inputB)))
      println(s"|**`$inputA`**|${types.map(dt => s"`${dt.toString}`").mkString("|")}|")
    }
  }
```

## How was this patch tested?

Unit tests added in `ParquetPartitionDiscoverySuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #19389 from HyukjinKwon/partition-type-coercion.
2017-11-21 20:53:38 +01:00
gatorsmile 96e947ed6c [SPARK-22569][SQL] Clean usage of addMutableState and splitExpressions
## What changes were proposed in this pull request?
This PR is to clean the usage of addMutableState and splitExpressions

1. replace hardcoded type string to ctx.JAVA_BOOLEAN etc.
2. create a default value of the initCode for ctx.addMutableStats
3. Use named arguments when calling `splitExpressions `

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19790 from gatorsmile/codeClean.
2017-11-21 13:48:09 +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 3c3eebc873 [SPARK-20101][SQL] Use OffHeapColumnVector when "spark.sql.columnVector.offheap.enable" is set to "true"
This PR enables to use ``OffHeapColumnVector`` when ``spark.sql.columnVector.offheap.enable`` is set to ``true``. While ``ColumnVector`` has two implementations ``OnHeapColumnVector`` and ``OffHeapColumnVector``, only ``OnHeapColumnVector`` is always used.

This PR implements the followings
- Pass ``OffHeapColumnVector`` to ``ColumnarBatch.allocate()`` when ``spark.sql.columnVector.offheap.enable`` is set to ``true``
- Free all of off-heap memory regions by ``OffHeapColumnVector.close()``
- Ensure to call ``OffHeapColumnVector.close()``

Use existing tests

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

Closes #17436 from kiszk/SPARK-20101.
2017-11-20 12:40:26 +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
Wenchen Fan b9dcbe5e1b [SPARK-22542][SQL] remove unused features in ColumnarBatch
## What changes were proposed in this pull request?

`ColumnarBatch` provides features to do fast filter and project in a columnar fashion, however this feature is never used by Spark, as Spark uses whole stage codegen and processes the data in a row fashion. This PR proposes to remove these unused features as we won't switch to columnar execution in the near future. Even we do, I think this part needs a proper redesign.

This is also a step to make `ColumnVector` public, as we don't wanna expose these features to users.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19766 from cloud-fan/vector.
2017-11-16 18:23:00 -08: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
Zhenhua Wang 11b60af737 [SPARK-17074][SQL] Generate equi-height histogram in column statistics
## What changes were proposed in this pull request?

Equi-height histogram is effective in cardinality estimation, and more accurate than basic column stats (min, max, ndv, etc) especially in skew distribution. So we need to support it.

For equi-height histogram, all buckets (intervals) have the same height (frequency).
In this PR, we use a two-step method to generate an equi-height histogram:
1. use `ApproximatePercentile` to get percentiles `p(0), p(1/n), p(2/n) ... p((n-1)/n), p(1)`;
2. construct range values of buckets, e.g. `[p(0), p(1/n)], [p(1/n), p(2/n)] ... [p((n-1)/n), p(1)]`, and use `ApproxCountDistinctForIntervals` to count ndv in each bucket. Each bucket is of the form: `(lowerBound, higherBound, ndv)`.

## How was this patch tested?

Added new test cases and modified some existing test cases.

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

Closes #19479 from wzhfy/generate_histogram.
2017-11-14 16:41:43 +01:00
hyukjinkwon 673c670465 [SPARK-17310][SQL] Add an option to disable record-level filter in Parquet-side
## What changes were proposed in this pull request?

There is a concern that Spark-side codegen row-by-row filtering might be faster than Parquet's one in general due to type-boxing and additional fuction calls which Spark's one tries to avoid.

So, this PR adds an option to disable/enable record-by-record filtering in Parquet side.

It sets the default to `false` to take the advantage of the improvement.

This was also discussed in https://github.com/apache/spark/pull/14671.
## How was this patch tested?

Manually benchmarks were performed. I generated a billion (1,000,000,000) records and tested equality comparison concatenated with `OR`. This filter combinations were made from 5 to 30.

It seem indeed Spark-filtering is faster in the test case and the gap increased as the filter tree becomes larger.

The details are as below:

**Code**

``` scala
test("Parquet-side filter vs Spark-side filter - record by record") {
  withTempPath { path =>
    val N = 1000 * 1000 * 1000
    val df = spark.range(N).toDF("a")
    df.write.parquet(path.getAbsolutePath)

    val benchmark = new Benchmark("Parquet-side vs Spark-side", N)
    Seq(5, 10, 20, 30).foreach { num =>
      val filterExpr = (0 to num).map(i => s"a = $i").mkString(" OR ")

      benchmark.addCase(s"Parquet-side filter - number of filters [$num]", 3) { _ =>
        withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> false.toString,
          SQLConf.PARQUET_RECORD_FILTER_ENABLED.key -> true.toString) {

          // We should strip Spark-side filter to compare correctly.
          stripSparkFilter(
            spark.read.parquet(path.getAbsolutePath).filter(filterExpr)).count()
        }
      }

      benchmark.addCase(s"Spark-side filter - number of filters [$num]", 3) { _ =>
        withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> false.toString,
          SQLConf.PARQUET_RECORD_FILTER_ENABLED.key -> false.toString) {

          spark.read.parquet(path.getAbsolutePath).filter(filterExpr).count()
        }
      }
    }

    benchmark.run()
  }
}
```

**Result**

```
Parquet-side vs Spark-side:              Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Parquet-side filter - number of filters [5]      4268 / 4367        234.3           4.3       0.8X
Spark-side filter - number of filters [5]      3709 / 3741        269.6           3.7       0.9X
Parquet-side filter - number of filters [10]      5673 / 5727        176.3           5.7       0.6X
Spark-side filter - number of filters [10]      3588 / 3632        278.7           3.6       0.9X
Parquet-side filter - number of filters [20]      8024 / 8440        124.6           8.0       0.4X
Spark-side filter - number of filters [20]      3912 / 3946        255.6           3.9       0.8X
Parquet-side filter - number of filters [30]    11936 / 12041         83.8          11.9       0.3X
Spark-side filter - number of filters [30]      3929 / 3978        254.5           3.9       0.8X
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15049 from HyukjinKwon/SPARK-17310.
2017-11-14 12:34:21 +01: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
Wenchen Fan 21a7bfd5c3 [SPARK-10365][SQL] Support Parquet logical type TIMESTAMP_MICROS
## What changes were proposed in this pull request?

This PR makes Spark to be able to read Parquet TIMESTAMP_MICROS values, and add a new config to allow Spark to write timestamp values to parquet as TIMESTAMP_MICROS type.

## How was this patch tested?

new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19702 from cloud-fan/parquet.
2017-11-11 22:40:26 +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
Liang-Chi Hsieh 87343e1556 [SPARK-22446][SQL][ML] Declare StringIndexerModel indexer udf as nondeterministic
## What changes were proposed in this pull request?

UDFs that can cause runtime exception on invalid data are not safe to pushdown, because its behavior depends on its position in the query plan. Pushdown of it will risk to change its original behavior.

The example reported in the JIRA and taken as test case shows this issue. We should declare UDFs that can cause runtime exception on invalid data as non-determinstic.

This updates the document of `deterministic` property in `Expression` and states clearly an UDF that can cause runtime exception on some specific input, should be declared as non-determinstic.

## How was this patch tested?

Added test. Manually test.

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

Closes #19662 from viirya/SPARK-22446.
2017-11-08 12:17:52 +01:00
Wenchen Fan d5202259d9 [SPARK-21127][SQL][FOLLOWUP] fix a config name typo
## What changes were proposed in this pull request?

`spark.sql.statistics.autoUpdate.size` should be `spark.sql.statistics.size.autoUpdate.enabled`. The previous name is confusing as users may treat it as a size config.

This config is in master branch only, no backward compatibility issue.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19667 from cloud-fan/minor.
2017-11-07 09:33:52 -08:00
Wenchen Fan 472db58cb1 [SPARK-22445][SQL] move CodegenContext.copyResult to CodegenSupport
## What changes were proposed in this pull request?

`CodegenContext.copyResult` is kind of a global status for whole stage codegen. But the tricky part is, it is only used to transfer an information from child to parent when calling the `consume` chain. We have to be super careful in `produce`/`consume`, to set it to true when producing multiple result rows, and set it to false in operators that start new pipeline(like sort).

This PR moves the `copyResult` to `CodegenSupport`, and call it at `WholeStageCodegenExec`. This is much easier to reason about.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19656 from cloud-fan/whole-sage.
2017-11-06 15:10:44 +01: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
Zhenhua Wang 59589bc654 [SPARK-22310][SQL] Refactor join estimation to incorporate estimation logic for different kinds of statistics
## What changes were proposed in this pull request?

The current join estimation logic is only based on basic column statistics (such as ndv, etc). If we want to add estimation for other kinds of statistics (such as histograms), it's not easy to incorporate into the current algorithm:
1. When we have multiple pairs of join keys, the current algorithm computes cardinality in a single formula. But if different join keys have different kinds of stats, the computation logic for each pair of join keys become different, so the previous formula does not apply.
2. Currently it computes cardinality and updates join keys' column stats separately. It's better to do these two steps together, since both computation and update logic are different for different kinds of stats.

## How was this patch tested?

Only refactor, covered by existing tests.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #19531 from wzhfy/join_est_refactor.
2017-10-31 11:13:48 +01:00
Wenchen Fan 079a2609d7 [SPARK-17788][SPARK-21033][SQL] fix the potential OOM in UnsafeExternalSorter and ShuffleExternalSorter
## What changes were proposed in this pull request?

In `UnsafeInMemorySorter`, one record may take 32 bytes: 1 `long` for pointer, 1 `long` for key-prefix, and another 2 `long`s as the temporary buffer for radix sort.

In `UnsafeExternalSorter`, we set the `DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD` to be `1024 * 1024 * 1024 / 2`, and hoping the max size of point array to be 8 GB. However this is wrong, `1024 * 1024 * 1024 / 2 * 32` is actually 16 GB, and if we grow the point array before reach this limitation, we may hit the max-page-size error.

Users may see exception like this on large dataset:
```
Caused by: java.lang.IllegalArgumentException: Cannot allocate a page with more than 17179869176 bytes
at org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:241)
at org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:121)
at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPageIfNecessary(UnsafeExternalSorter.java:374)
at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.insertRecord(UnsafeExternalSorter.java:396)
at org.apache.spark.sql.execution.UnsafeExternalRowSorter.insertRow(UnsafeExternalRowSorter.java:94)
...
```

Setting `DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD` to a smaller number is not enough, users can still set the config to a big number and trigger the too large page size issue. This PR fixes it by explicitly handling the too large page size exception in the sorter and spill.

This PR also change the type of `spark.shuffle.spill.numElementsForceSpillThreshold` to int, because it's only compared with `numRecords`, which is an int. This is an internal conf so we don't have a serious compatibility issue.

## How was this patch tested?

TODO

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18251 from cloud-fan/sort.
2017-10-30 17:53:06 +01:00
Wenchen Fan 9f02d7dc53 [SPARK-22385][SQL] MapObjects should not access list element by index
## What changes were proposed in this pull request?

This issue was discovered and investigated by Ohad Raviv and Sean Owen in https://issues.apache.org/jira/browse/SPARK-21657. The input data of `MapObjects` may be a `List` which has O(n) complexity for accessing by index. When converting input data to catalyst array, `MapObjects` gets element by index in each loop, and results to bad performance.

This PR fixes this issue by accessing elements via Iterator.

## How was this patch tested?

using the test script in https://issues.apache.org/jira/browse/SPARK-21657
```
val BASE = 100000000
val N = 100000
val df = sc.parallelize(List(("1234567890", (BASE to (BASE+N)).map(x => (x.toString, (x+1).toString, (x+2).toString, (x+3).toString)).toList ))).toDF("c1", "c_arr")
spark.time(df.queryExecution.toRdd.foreach(_ => ()))
```

We can see 50x speed up.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19603 from cloud-fan/map-objects.
2017-10-30 11:00:44 +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
Takuya UESHIN 4c5269f1aa [SPARK-22370][SQL][PYSPARK] Config values should be captured in Driver.
## What changes were proposed in this pull request?

`ArrowEvalPythonExec` and `FlatMapGroupsInPandasExec` are refering config values of `SQLConf` in function for `mapPartitions`/`mapPartitionsInternal`, but we should capture them in Driver.

## How was this patch tested?

Added a test and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #19587 from ueshin/issues/SPARK-22370.
2017-10-28 18:33:09 +01:00
Reynold Xin d28d5732ae [SPARK-21619][SQL] Fail the execution of canonicalized plans explicitly
## What changes were proposed in this pull request?
Canonicalized plans are not supposed to be executed. I ran into a case in which there's some code that accidentally calls execute on a canonicalized plan. This patch throws a more explicit exception when that happens.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #18828 from rxin/SPARK-21619.
2017-10-27 23:44:24 -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
gatorsmile 36b826f5d1 [TRIVIAL][SQL] Code cleaning in ResolveReferences
## What changes were proposed in this pull request?
This PR is to clean the related codes majorly based on the today's code review on  https://github.com/apache/spark/pull/19559

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19585 from gatorsmile/trivialFixes.
2017-10-27 07:52:10 -07:00
Jose Torres 8e9863531b [SPARK-22366] Support ignoring missing files
## What changes were proposed in this pull request?

Add a flag "spark.sql.files.ignoreMissingFiles" to parallel the existing flag "spark.sql.files.ignoreCorruptFiles".

## How was this patch tested?

new unit test

Author: Jose Torres <jose@databricks.com>

Closes #19581 from joseph-torres/SPARK-22366.
2017-10-26 16:55:30 -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
gatorsmile d8cada8d1d [SPARK-20331][SQL][FOLLOW-UP] Add a SQLConf for enhanced Hive partition pruning predicate pushdown
## What changes were proposed in this pull request?
This is a follow-up PR of https://github.com/apache/spark/pull/17633.

This PR is to add a conf `spark.sql.hive.advancedPartitionPredicatePushdown.enabled`, which can be used to turn the enhancement off.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19547 from gatorsmile/Spark20331FollowUp.
2017-10-21 10:05:45 -07:00
Zhenhua Wang d9f286d261 [SPARK-22326][SQL] Remove unnecessary hashCode and equals methods
## What changes were proposed in this pull request?

Plan equality should be computed by `canonicalized`, so we can remove unnecessary `hashCode` and `equals` methods.

## How was this patch tested?

Existing tests.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #19539 from wzhfy/remove_equals.
2017-10-20 20:58:55 -07:00
Takuya UESHIN b8624b06e5 [SPARK-20396][SQL][PYSPARK][FOLLOW-UP] groupby().apply() with pandas udf
## What changes were proposed in this pull request?

This is a follow-up of #18732.
This pr modifies `GroupedData.apply()` method to convert pandas udf to grouped udf implicitly.

## How was this patch tested?

Exisiting tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #19517 from ueshin/issues/SPARK-20396/fup2.
2017-10-20 12:44:30 -07:00
CenYuhai 16c9cc68c5 [SPARK-21055][SQL] replace grouping__id with grouping_id()
## What changes were proposed in this pull request?
spark does not support grouping__id, it has grouping_id() instead.
But it is not convenient for hive user to change to spark-sql
so this pr is to replace grouping__id with grouping_id()
hive user need not to alter their scripts

## How was this patch tested?

test with SQLQuerySuite.scala

Author: CenYuhai <yuhai.cen@ele.me>

Closes #18270 from cenyuhai/SPARK-21055.
2017-10-20 09:27:39 -07:00
maryannxue 72561ecf4b [SPARK-22266][SQL] The same aggregate function was evaluated multiple times
## What changes were proposed in this pull request?

To let the same aggregate function that appear multiple times in an Aggregate be evaluated only once, we need to deduplicate the aggregate expressions. The original code was trying to use a "distinct" call to get a set of aggregate expressions, but did not work, since the "distinct" did not compare semantic equality. And even if it did, further work should be done in result expression rewriting.
In this PR, I changed the "set" to a map mapping the semantic identity of a aggregate expression to itself. Thus, later on, when rewriting result expressions (i.e., output expressions), the aggregate expression reference can be fixed.

## How was this patch tested?

Added a new test in SQLQuerySuite

Author: maryannxue <maryann.xue@gmail.com>

Closes #19488 from maryannxue/spark-22266.
2017-10-18 20:59:40 +08:00
Huaxin Gao 28f9f3f225 [SPARK-22271][SQL] mean overflows and returns null for some decimal variables
## What changes were proposed in this pull request?

In Average.scala, it has
```
  override lazy val evaluateExpression = child.dataType match {
    case DecimalType.Fixed(p, s) =>
      // increase the precision and scale to prevent precision loss
      val dt = DecimalType.bounded(p + 14, s + 4)
      Cast(Cast(sum, dt) / Cast(count, dt), resultType)
    case _ =>
      Cast(sum, resultType) / Cast(count, resultType)
  }

  def setChild (newchild: Expression) = {
    child = newchild
  }

```
It is possible that  Cast(count, dt), resultType) will make the precision of the decimal number bigger than 38, and this causes over flow.  Since count is an integer and doesn't need a scale, I will cast it using DecimalType.bounded(38,0)
## How was this patch tested?
In DataFrameSuite, I will add a test case.

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

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

Closes #19496 from huaxingao/spark-22271.
2017-10-17 12:50:41 -07:00
Burak Yavuz e8547ffb49 [SPARK-22238] Fix plan resolution bug caused by EnsureStatefulOpPartitioning
## What changes were proposed in this pull request?

In EnsureStatefulOpPartitioning, we check that the inputRDD to a SparkPlan has the expected partitioning for Streaming Stateful Operators. The problem is that we are not allowed to access this information during planning.
The reason we added that check was because CoalesceExec could actually create RDDs with 0 partitions. We should fix it such that when CoalesceExec says that there is a SinglePartition, there is in fact an inputRDD of 1 partition instead of 0 partitions.

## How was this patch tested?

Regression test in StreamingQuerySuite

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #19467 from brkyvz/stateful-op.
2017-10-14 17:39:15 -07:00
Wenchen Fan 3823dc88d3 [SPARK-22252][SQL][FOLLOWUP] Command should not be a LeafNode
## What changes were proposed in this pull request?

This is a minor folllowup of #19474 .

#19474 partially reverted #18064 but accidentally introduced a behavior change. `Command` extended `LogicalPlan` before #18064 , but #19474 made it extend `LeafNode`. This is an internal behavior change as now all `Command` subclasses can't define children, and they have to implement `computeStatistic` method.

This PR fixes this by making `Command` extend `LogicalPlan`

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19493 from cloud-fan/minor.
2017-10-13 10:49:48 -07: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
Wang Gengliang 3ff766f61a [SPARK-22263][SQL] Refactor deterministic as lazy value
## What changes were proposed in this pull request?

The method `deterministic` is frequently called in optimizer.
Refactor `deterministic` as lazy value, in order to avoid redundant computations.

## How was this patch tested?
Simple benchmark test over TPC-DS queries, run time from query string to optimized plan(continuous  20 runs, and get the average of last 5 results):
Before changes: 12601 ms
After changes: 11993ms
This is 4.8% performance improvement.

Also run test with Unit test.

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #19478 from gengliangwang/deterministicAsLazyVal.
2017-10-12 18:47:16 -07:00
Steve Loughran 9104add4c7 [SPARK-22217][SQL] ParquetFileFormat to support arbitrary OutputCommitters
## What changes were proposed in this pull request?

`ParquetFileFormat` to relax its requirement of output committer class from `org.apache.parquet.hadoop.ParquetOutputCommitter` or subclass thereof (and so implicitly Hadoop `FileOutputCommitter`) to any committer implementing `org.apache.hadoop.mapreduce.OutputCommitter`

This enables output committers which don't write to the filesystem the way `FileOutputCommitter` does to save parquet data from a dataframe: at present you cannot do this.

Before a committer which isn't a subclass of `ParquetOutputCommitter`, it checks to see if the context has requested summary metadata by setting `parquet.enable.summary-metadata`. If true, and the committer class isn't a parquet committer, it raises a RuntimeException with an error message.

(It could downgrade, of course, but raising an exception makes it clear there won't be an summary. It also makes the behaviour testable.)

Note that `SQLConf` already states that any `OutputCommitter` can be used, but that typically it's a subclass of ParquetOutputCommitter. That's not currently true. This patch will make the code consistent with the docs, adding tests to verify,

## How was this patch tested?

The patch includes a test suite, `ParquetCommitterSuite`, with a new committer, `MarkingFileOutputCommitter` which extends `FileOutputCommitter` and writes a marker file in the destination directory. The presence of the marker file can be used to verify the new committer was used. The tests then try the combinations of Parquet committer summary/no-summary and marking committer summary/no-summary.

| committer | summary | outcome |
|-----------|---------|---------|
| parquet   | true    | success |
| parquet   | false   | success |
| marking   | false   | success with marker |
| marking   | true    | exception |

All tests are happy.

Author: Steve Loughran <stevel@hortonworks.com>

Closes #19448 from steveloughran/cloud/SPARK-22217-committer.
2017-10-13 08:40:26 +09:00
Wenchen Fan 274f0efefa [SPARK-22252][SQL] FileFormatWriter should respect the input query schema
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/18064, we allowed `RunnableCommand` to have children in order to fix some UI issues. Then we made `InsertIntoXXX` commands take the input `query` as a child, when we do the actual writing, we just pass the physical plan to the writer(`FileFormatWriter.write`).

However this is problematic. In Spark SQL, optimizer and planner are allowed to change the schema names a little bit. e.g. `ColumnPruning` rule will remove no-op `Project`s, like `Project("A", Scan("a"))`, and thus change the output schema from "<A: int>" to `<a: int>`. When it comes to writing, especially for self-description data format like parquet, we may write the wrong schema to the file and cause null values at the read path.

Fortunately, in https://github.com/apache/spark/pull/18450 , we decided to allow nested execution and one query can map to multiple executions in the UI. This releases the major restriction in #18604 , and now we don't have to take the input `query` as child of `InsertIntoXXX` commands.

So the fix is simple, this PR partially revert #18064 and make `InsertIntoXXX` commands leaf nodes again.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19474 from cloud-fan/bug.
2017-10-12 20:20:44 +08: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
Marcelo Vanzin bd4eb9ce57 [SPARK-19558][SQL] Add config key to register QueryExecutionListeners automatically.
This change adds a new SQL config key that is equivalent to SparkContext's
"spark.extraListeners", allowing users to register QueryExecutionListener
instances through the Spark configuration system instead of having to
explicitly do it in code.

The code used by SparkContext to implement the feature was refactored into
a helper method in the Utils class, and SQL's ExecutionListenerManager was
modified to use it to initialize listener declared in the configuration.

Unit tests were added to verify all the new functionality.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19309 from vanzin/SPARK-19558.
2017-10-10 15:50:37 -07:00
Li Jin bfc7e1fe1a [SPARK-20396][SQL][PYSPARK] groupby().apply() with pandas udf
## What changes were proposed in this pull request?

This PR adds an apply() function on df.groupby(). apply() takes a pandas udf that is a transformation on `pandas.DataFrame` -> `pandas.DataFrame`.

Static schema
-------------------
```
schema = df.schema

pandas_udf(schema)
def normalize(df):
    df = df.assign(v1 = (df.v1 - df.v1.mean()) / df.v1.std()
    return df

df.groupBy('id').apply(normalize)
```
Dynamic schema
-----------------------
**This use case is removed from the PR and we will discuss this as a follow up. See discussion https://github.com/apache/spark/pull/18732#pullrequestreview-66583248**

Another example to use pd.DataFrame dtypes as output schema of the udf:

```
sample_df = df.filter(df.id == 1).toPandas()

def foo(df):
      ret = # Some transformation on the input pd.DataFrame
      return ret

foo_udf = pandas_udf(foo, foo(sample_df).dtypes)

df.groupBy('id').apply(foo_udf)
```
In interactive use case, user usually have a sample pd.DataFrame to test function `foo` in their notebook. Having been able to use `foo(sample_df).dtypes` frees user from specifying the output schema of `foo`.

Design doc: https://github.com/icexelloss/spark/blob/pandas-udf-doc/docs/pyspark-pandas-udf.md

## How was this patch tested?
* Added GroupbyApplyTest

Author: Li Jin <ice.xelloss@gmail.com>
Author: Takuya UESHIN <ueshin@databricks.com>
Author: Bryan Cutler <cutlerb@gmail.com>

Closes #18732 from icexelloss/groupby-apply-SPARK-20396.
2017-10-11 07:32:01 +09: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
Ryan Blue 155ab6347e [SPARK-22170][SQL] Reduce memory consumption in broadcast joins.
## What changes were proposed in this pull request?

This updates the broadcast join code path to lazily decompress pages and
iterate through UnsafeRows to prevent all rows from being held in memory
while the broadcast table is being built.

## How was this patch tested?

Existing tests.

Author: Ryan Blue <blue@apache.org>

Closes #19394 from rdblue/broadcast-driver-memory.
2017-10-09 15:22:41 -07:00
Liang-Chi Hsieh debcbec749 [SPARK-21947][SS] Check and report error when monotonically_increasing_id is used in streaming query
## What changes were proposed in this pull request?

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

## How was this patch tested?

Added test.

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

Closes #19336 from viirya/SPARK-21947.
2017-10-06 13:10:04 -07:00
Xingbo Jiang 08b204fd2c [SPARK-22214][SQL] Refactor the list hive partitions code
## What changes were proposed in this pull request?

In this PR we make a few changes to the list hive partitions code, to make the code more extensible.
The following changes are made:
1. In `HiveClientImpl.getPartitions()`, call `client.getPartitions` instead of `shim.getAllPartitions` when `spec` is empty;
2. In `HiveTableScanExec`, previously we always call `listPartitionsByFilter` if the config `metastorePartitionPruning` is enabled, but actually, we'd better call `listPartitions` if `partitionPruningPred` is empty;
3.  We should use sessionCatalog instead of SharedState.externalCatalog in `HiveTableScanExec`.

## How was this patch tested?

Tested by existing test cases since this is code refactor, no regression or behavior change is expected.

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

Closes #19444 from jiangxb1987/hivePartitions.
2017-10-06 12:53:35 -07:00
Wenchen Fan bb035f1ee5 [SPARK-22169][SQL] support byte length literal as identifier
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

regression test

Author: Wenchen Fan <wenchen@databricks.com>

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

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

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

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

## How was this patch tested?

new unit tests

Author: Jose Torres <jose@databricks.com>

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

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

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

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19395 from gatorsmile/desc.
2017-10-03 21:27:58 -07:00
Reynold Xin 4c5158eec9 [SPARK-21644][SQL] LocalLimit.maxRows is defined incorrectly
## What changes were proposed in this pull request?
The definition of `maxRows` in `LocalLimit` operator was simply wrong. This patch introduces a new `maxRowsPerPartition` method and uses that in pruning. The patch also adds more documentation on why we need local limit vs global limit.

Note that this previously has never been a bug because the way the code is structured, but future use of the maxRows could lead to bugs.

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

Closes #18851

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

Closes #19393 from gatorsmile/pr-18851.
2017-10-03 12:38:13 -07:00
gatorsmile 530fe68329 [SPARK-21904][SQL] Rename tempTables to tempViews in SessionCatalog
### What changes were proposed in this pull request?
`tempTables` is not right. To be consistent, we need to rename the internal variable names/comments to tempViews in SessionCatalog too.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19117 from gatorsmile/renameTempTablesToTempViews.
2017-09-29 19:35:32 -07:00
Wang Gengliang 0fa4dbe4f4 [SPARK-22141][FOLLOWUP][SQL] Add comments for the order of batches
## What changes were proposed in this pull request?
Add comments for specifying the position of  batch "Check Cartesian Products", as rxin suggested in https://github.com/apache/spark/pull/19362 .

## How was this patch tested?
Unit test

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #19379 from gengliangwang/SPARK-22141-followup.
2017-09-28 23:23:30 -07:00
Reynold Xin 323806e68f [SPARK-22160][SQL] Make sample points per partition (in range partitioner) configurable and bump the default value up to 100
## What changes were proposed in this pull request?
Spark's RangePartitioner hard codes the number of sampling points per partition to be 20. This is sometimes too low. This ticket makes it configurable, via spark.sql.execution.rangeExchange.sampleSizePerPartition, and raises the default in Spark SQL to be 100.

## How was this patch tested?
Added a pretty sophisticated test based on chi square test ...

Author: Reynold Xin <rxin@databricks.com>

Closes #19387 from rxin/SPARK-22160.
2017-09-28 21:07:12 -07:00
Reynold Xin d29d1e8799 [SPARK-22159][SQL] Make config names consistently end with "enabled".
## What changes were proposed in this pull request?
spark.sql.execution.arrow.enable and spark.sql.codegen.aggregate.map.twolevel.enable -> enabled

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #19384 from rxin/SPARK-22159.
2017-09-28 15:59:05 -07:00
Wang Gengliang 9c5935d00b [SPARK-22141][SQL] Propagate empty relation before checking Cartesian products
## What changes were proposed in this pull request?

When inferring constraints from children, Join's condition can be simplified as None.
For example,
```
val testRelation = LocalRelation('a.int)
val x = testRelation.as("x")
val y = testRelation.where($"a" === 2 && !($"a" === 2)).as("y")
x.join.where($"x.a" === $"y.a")
```
The plan will become
```
Join Inner
:- LocalRelation <empty>, [a#23]
+- LocalRelation <empty>, [a#224]
```
And the Cartesian products check will throw exception for above plan.

Propagate empty relation before checking Cartesian products, and the issue is resolved.

## How was this patch tested?

Unit test

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #19362 from gengliangwang/MoveCheckCartesianProducts.
2017-09-27 12:44:10 +02:00
Juliusz Sompolski f21f6ce998 [SPARK-22103][FOLLOWUP] Rename addExtraCode to addInnerClass
## What changes were proposed in this pull request?

Address PR comments that appeared post-merge, to rename `addExtraCode` to `addInnerClass`,
and not count the size of the inner class to the size of the outer class.

## How was this patch tested?

YOLO.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #19353 from juliuszsompolski/SPARK-22103followup.
2017-09-26 10:04:34 -07:00
Juliusz Sompolski 038b185736 [SPARK-22103] Move HashAggregateExec parent consume to a separate function in codegen
## What changes were proposed in this pull request?

HashAggregateExec codegen uses two paths for fast hash table and a generic one.
It generates code paths for iterating over both, and both code paths generate the consume code of the parent operator, resulting in that code being expanded twice.
This leads to a long generated function that might be an issue for the compiler (see e.g. SPARK-21603).
I propose to remove the double expansion by generating the consume code in a helper function that can just be called from both iterating loops.

An issue with separating the `consume` code to a helper function was that a number of places relied and assumed on being in the scope of an outside `produce` loop and e.g. use `continue` to jump out.
I replaced such code flows with nested scopes. It is code that should be handled the same by compiler, while getting rid of depending on assumptions that are outside of the `consume`'s own scope.

## How was this patch tested?

Existing test coverage.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #19324 from juliuszsompolski/aggrconsumecodegen.
2017-09-25 12:50:25 -07:00
Zhenhua Wang 365a29bdbf [SPARK-22100][SQL] Make percentile_approx support date/timestamp type and change the output type to be the same as input type
## What changes were proposed in this pull request?

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

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

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

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

## How was this patch tested?

Added a new test and modified some existing tests.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #19321 from wzhfy/approx_percentile_support_types.
2017-09-25 09:28:42 -07:00
Sean Owen 576c43fb42 [SPARK-22087][SPARK-14650][WIP][BUILD][REPL][CORE] Compile Spark REPL for Scala 2.12 + other 2.12 fixes
## What changes were proposed in this pull request?

Enable Scala 2.12 REPL. Fix most remaining issues with 2.12 compilation and warnings, including:

- Selecting Kafka 0.10.1+ for Scala 2.12 and patching over a minor API difference
- Fixing lots of "eta expansion of zero arg method deprecated" warnings
- Resolving the SparkContext.sequenceFile implicits compile problem
- Fixing an odd but valid jetty-server missing dependency in hive-thriftserver

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #19307 from srowen/Scala212.
2017-09-24 09:40:13 +01:00
Kevin Yu 4a8c9e29bc [SPARK-22110][SQL][DOCUMENTATION] Add usage and improve documentation with arguments and examples for trim function
## What changes were proposed in this pull request?

This PR proposes to enhance the documentation for `trim` functions in the function description session.

- Add more `usage`, `arguments` and `examples` for the trim function
- Adjust space in the `usage` session

After the changes, the trim function documentation will look like this:

- `trim`

```trim(str) - Removes the leading and trailing space characters from str.

trim(BOTH trimStr FROM str) - Remove the leading and trailing trimStr characters from str

trim(LEADING trimStr FROM str) - Remove the leading trimStr characters from str

trim(TRAILING trimStr FROM str) - Remove the trailing trimStr characters from str

Arguments:

str - a string expression
trimStr - the trim string characters to trim, the default value is a single space
BOTH, FROM - these are keywords to specify trimming string characters from both ends of the string
LEADING, FROM - these are keywords to specify trimming string characters from the left end of the string
TRAILING, FROM - these are keywords to specify trimming string characters from the right end of the string
Examples:

> SELECT trim('    SparkSQL   ');
 SparkSQL
> SELECT trim('SL', 'SSparkSQLS');
 parkSQ
> SELECT trim(BOTH 'SL' FROM 'SSparkSQLS');
 parkSQ
> SELECT trim(LEADING 'SL' FROM 'SSparkSQLS');
 parkSQLS
> SELECT trim(TRAILING 'SL' FROM 'SSparkSQLS');
 SSparkSQ
```

- `ltrim`

```ltrim

ltrim(str) - Removes the leading space characters from str.

ltrim(trimStr, str) - Removes the leading string contains the characters from the trim string

Arguments:

str - a string expression
trimStr - the trim string characters to trim, the default value is a single space
Examples:

> SELECT ltrim('    SparkSQL   ');
 SparkSQL
> SELECT ltrim('Sp', 'SSparkSQLS');
 arkSQLS
```

- `rtrim`
```rtrim

rtrim(str) - Removes the trailing space characters from str.

rtrim(trimStr, str) - Removes the trailing string which contains the characters from the trim string from the str

Arguments:

str - a string expression
trimStr - the trim string characters to trim, the default value is a single space
Examples:

> SELECT rtrim('    SparkSQL   ');
 SparkSQL
> SELECT rtrim('LQSa', 'SSparkSQLS');
 SSpark
```

This is the trim characters function jira: [trim function](https://issues.apache.org/jira/browse/SPARK-14878)

## How was this patch tested?

Manually tested
```
spark-sql> describe function extended trim;
17/09/22 17:03:04 INFO CodeGenerator: Code generated in 153.026533 ms
Function: trim
Class: org.apache.spark.sql.catalyst.expressions.StringTrim
Usage:
    trim(str) - Removes the leading and trailing space characters from `str`.

    trim(BOTH trimStr FROM str) - Remove the leading and trailing `trimStr` characters from `str`

    trim(LEADING trimStr FROM str) - Remove the leading `trimStr` characters from `str`

    trim(TRAILING trimStr FROM str) - Remove the trailing `trimStr` characters from `str`

Extended Usage:
    Arguments:
      * str - a string expression
      * trimStr - the trim string characters to trim, the default value is a single space
      * BOTH, FROM - these are keywords to specify trimming string characters from both ends of
          the string
      * LEADING, FROM - these are keywords to specify trimming string characters from the left
          end of the string
      * TRAILING, FROM - these are keywords to specify trimming string characters from the right
          end of the string

    Examples:
      > SELECT trim('    SparkSQL   ');
       SparkSQL
      > SELECT trim('SL', 'SSparkSQLS');
       parkSQ
      > SELECT trim(BOTH 'SL' FROM 'SSparkSQLS');
       parkSQ
      > SELECT trim(LEADING 'SL' FROM 'SSparkSQLS');
       parkSQLS
      > SELECT trim(TRAILING 'SL' FROM 'SSparkSQLS');
       SSparkSQ
  ```
```
spark-sql> describe function extended ltrim;
Function: ltrim
Class: org.apache.spark.sql.catalyst.expressions.StringTrimLeft
Usage:
    ltrim(str) - Removes the leading space characters from `str`.

    ltrim(trimStr, str) - Removes the leading string contains the characters from the trim string

Extended Usage:
    Arguments:
      * str - a string expression
      * trimStr - the trim string characters to trim, the default value is a single space

    Examples:
      > SELECT ltrim('    SparkSQL   ');
       SparkSQL
      > SELECT ltrim('Sp', 'SSparkSQLS');
       arkSQLS

```

```
spark-sql> describe function extended rtrim;
Function: rtrim
Class: org.apache.spark.sql.catalyst.expressions.StringTrimRight
Usage:
    rtrim(str) - Removes the trailing space characters from `str`.

    rtrim(trimStr, str) - Removes the trailing string which contains the characters from the trim string from the `str`

Extended Usage:
    Arguments:
      * str - a string expression
      * trimStr - the trim string characters to trim, the default value is a single space

    Examples:
      > SELECT rtrim('    SparkSQL   ');
       SparkSQL
      > SELECT rtrim('LQSa', 'SSparkSQLS');
       SSpark

```

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

Closes #19329 from kevinyu98/spark-14878-5.
2017-09-23 10:27:40 -07:00
Sean Owen 50ada2a4d3 [SPARK-22033][CORE] BufferHolder, other size checks should account for the specific VM array size limitations
## What changes were proposed in this pull request?

Try to avoid allocating an array bigger than Integer.MAX_VALUE - 8, which is the actual max size on some JVMs, in several places

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #19266 from srowen/SPARK-22033.
2017-09-23 15:40:59 +01:00
maryannxue 5960686e79 [SPARK-21998][SQL] SortMergeJoinExec did not calculate its outputOrdering correctly during physical planning
## What changes were proposed in this pull request?

Right now the calculation of SortMergeJoinExec's outputOrdering relies on the fact that its children have already been sorted on the join keys, while this is often not true until EnsureRequirements has been applied. So we ended up not getting the correct outputOrdering during physical planning stage before Sort nodes are added to the children.

For example, J = {A join B on key1 = key2}
1. if A is NOT ordered on key1 ASC, J's outputOrdering should include "key1 ASC"
2. if A is ordered on key1 ASC, J's outputOrdering should include "key1 ASC"
3. if A is ordered on key1 ASC, with sameOrderExp=c1, J's outputOrdering should include "key1 ASC, sameOrderExp=c1"

So to fix this I changed the  behavior of <code>getKeyOrdering(keys, childOutputOrdering)</code> to:
1. If the childOutputOrdering satisfies (is a superset of) the required child ordering => childOutputOrdering
2. Otherwise => required child ordering

In addition, I organized the logic for deciding the relationship between two orderings into SparkPlan, so that it can be reused by EnsureRequirements and SortMergeJoinExec, and potentially other classes.

## How was this patch tested?

Added new test cases.
Passed all integration tests.

Author: maryannxue <maryann.xue@gmail.com>

Closes #19281 from maryannxue/spark-21998.
2017-09-21 23:54:16 -07:00
Tathagata Das f32a842505 [SPARK-22053][SS] Stream-stream inner join in Append Mode
## What changes were proposed in this pull request?

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

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

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

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

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

Refer to the scaladocs class for more implementation details.

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

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

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

Closes #19271 from tdas/SPARK-22053.
2017-09-21 15:39:07 -07:00
Liang-Chi Hsieh 9cac249fd5 [SPARK-22088][SQL] Incorrect scalastyle comment causes wrong styles in stringExpressions
## What changes were proposed in this pull request?

There is an incorrect `scalastyle:on` comment in `stringExpressions.scala` and causes the line size limit check ineffective in the file. There are many lines of code and comment which are more than 100 chars.

## How was this patch tested?

Code style change only.

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

Closes #19305 from viirya/fix-wrong-style.
2017-09-21 11:51:00 -07:00
Liang-Chi Hsieh 1270e71753 [SPARK-22086][DOCS] Add expression description for CASE WHEN
## What changes were proposed in this pull request?

In SQL conditional expressions, only CASE WHEN lacks for expression description. This patch fills the gap.

## How was this patch tested?

Only documentation change.

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

Closes #19304 from viirya/casewhen-doc.
2017-09-21 22:45:06 +09:00
Zhenhua Wang 1d1a09be9f [SPARK-17997][SQL] Add an aggregation function for counting distinct values for multiple intervals
## What changes were proposed in this pull request?

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

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

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

## How was this patch tested?

Add new test cases.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #15544 from wzhfy/countIntervals.
2017-09-21 21:43:02 +08:00
Wenchen Fan 352bea5457 [SPARK-22076][SQL][FOLLOWUP] Expand.projections should not be a Stream
## What changes were proposed in this pull request?

This a follow-up of https://github.com/apache/spark/pull/19289 , we missed another place: `rollup`. `Seq.init.toSeq` also returns a `Stream`, we should fix it too.

## How was this patch tested?

manually

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19298 from cloud-fan/bug.
2017-09-20 21:13:46 -07:00
Wenchen Fan ce6a71e013 [SPARK-22076][SQL] Expand.projections should not be a Stream
## What changes were proposed in this pull request?

Spark with Scala 2.10 fails with a group by cube:
```
spark.range(1).select($"id" as "a", $"id" as "b").write.partitionBy("a").mode("overwrite").saveAsTable("rollup_bug")
spark.sql("select 1 from rollup_bug group by rollup ()").show
```

It can be traced back to https://github.com/apache/spark/pull/15484 , which made `Expand.projections` a lazy `Stream` for group by cube.

In scala 2.10 `Stream` captures a lot of stuff, and in this case it captures the entire query plan which has some un-serializable parts.

This change is also good for master branch, to reduce the serialized size of `Expand.projections`.

## How was this patch tested?

manually verified with Spark with Scala 2.10.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19289 from cloud-fan/bug.
2017-09-20 09:00:43 -07:00
Sean Owen e17901d6df [SPARK-22049][DOCS] Confusing behavior of from_utc_timestamp and to_utc_timestamp
## What changes were proposed in this pull request?

Clarify behavior of to_utc_timestamp/from_utc_timestamp with an example

## How was this patch tested?

Doc only change / existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #19276 from srowen/SPARK-22049.
2017-09-20 20:47:17 +09:00
aokolnychyi ee13f3e3dc [SPARK-21969][SQL] CommandUtils.updateTableStats should call refreshTable
## What changes were proposed in this pull request?

Tables in the catalog cache are not invalidated once their statistics are updated. As a consequence, existing sessions will use the cached information even though it is not valid anymore. Consider and an example below.

```
// step 1
spark.range(100).write.saveAsTable("tab1")
// step 2
spark.sql("analyze table tab1 compute statistics")
// step 3
spark.sql("explain cost select distinct * from tab1").show(false)
// step 4
spark.range(100).write.mode("append").saveAsTable("tab1")
// step 5
spark.sql("explain cost select distinct * from tab1").show(false)
```

After step 3, the table will be present in the catalog relation cache. Step 4 will correctly update the metadata inside the catalog but will NOT invalidate the cache.

By the way, ``spark.sql("analyze table tab1 compute statistics")`` between step 3 and step 4 would also solve the problem.

## How was this patch tested?

Current and additional unit tests.

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

Closes #19252 from aokolnychyi/spark-21969.
2017-09-19 14:19:13 -07:00
Armin 7c92351f43 [MINOR][CORE] Cleanup dead code and duplication in Mem. Management
## What changes were proposed in this pull request?

* Removed the method `org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter#alignToWords`.
It became unused as a result of 85b0a15754
(SPARK-15962) introducing word alignment for unsafe arrays.
* Cleaned up duplicate code in memory management and unsafe sorters
  * The change extracting the exception paths is more than just cosmetics since it def. reduces the size the affected methods compile to

## How was this patch tested?

* Build still passes after removing the method, grepping the codebase for `alignToWords` shows no reference to it anywhere either.
* Dried up code is covered by existing tests.

Author: Armin <me@obrown.io>

Closes #19254 from original-brownbear/cleanup-mem-consumer.
2017-09-19 10:06:32 +01:00
Kevin Yu c66d64b3df [SPARK-14878][SQL] Trim characters string function support
#### What changes were proposed in this pull request?

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

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

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

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

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

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

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

## How was this patch tested?
New unit test

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

Closes #19240 from tdas/SPARK-22018.
2017-09-14 22:32:16 -07:00
goldmedal a28728a9af [SPARK-21513][SQL][FOLLOWUP] Allow UDF to_json support converting MapType to json for PySpark and SparkR
## What changes were proposed in this pull request?
In previous work SPARK-21513, we has allowed `MapType` and `ArrayType` of `MapType`s convert to a json string but only for Scala API. In this follow-up PR, we will make SparkSQL support it for PySpark and SparkR, too. We also fix some little bugs and comments of the previous work in this follow-up PR.

### For PySpark
```
>>> data = [(1, {"name": "Alice"})]
>>> df = spark.createDataFrame(data, ("key", "value"))
>>> df.select(to_json(df.value).alias("json")).collect()
[Row(json=u'{"name":"Alice")']
>>> data = [(1, [{"name": "Alice"}, {"name": "Bob"}])]
>>> df = spark.createDataFrame(data, ("key", "value"))
>>> df.select(to_json(df.value).alias("json")).collect()
[Row(json=u'[{"name":"Alice"},{"name":"Bob"}]')]
```
### For SparkR
```
# Converts a map into a JSON object
df2 <- sql("SELECT map('name', 'Bob')) as people")
df2 <- mutate(df2, people_json = to_json(df2$people))
# Converts an array of maps into a JSON array
df2 <- sql("SELECT array(map('name', 'Bob'), map('name', 'Alice')) as people")
df2 <- mutate(df2, people_json = to_json(df2$people))
```
## How was this patch tested?
Add unit test cases.

cc viirya HyukjinKwon

Author: goldmedal <liugs963@gmail.com>

Closes #19223 from goldmedal/SPARK-21513-fp-PySaprkAndSparkR.
2017-09-15 11:53:10 +09:00
donnyzone 21c4450fb2 [SPARK-21980][SQL] References in grouping functions should be indexed with semanticEquals
## What changes were proposed in this pull request?

https://issues.apache.org/jira/browse/SPARK-21980

This PR fixes the issue in ResolveGroupingAnalytics rule, which indexes the column references in grouping functions without considering case sensitive configurations.

The problem can be reproduced by:

`val df = spark.createDataFrame(Seq((1, 1), (2, 1), (2, 2))).toDF("a", "b")
 df.cube("a").agg(grouping("A")).show()`

## How was this patch tested?
unit tests

Author: donnyzone <wellfengzhu@gmail.com>

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

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

cc viirya HyukjinKwon

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

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

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

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

## How was this patch tested?

Unit test

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #19201 from gengliangwang/QueryPlanConstraints.
2017-09-12 13:02:29 -07:00
Zhenhua Wang 515910e9bd [SPARK-17642][SQL] support DESC EXTENDED/FORMATTED table column commands
## What changes were proposed in this pull request?

Support DESC (EXTENDED | FORMATTED) ? TABLE COLUMN command.
Support DESC EXTENDED | FORMATTED TABLE COLUMN command to show column-level statistics.
Do NOT support describe nested columns.

## How was this patch tested?

Added test cases.

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

Closes #16422 from wzhfy/descColumn.
2017-09-12 08:59:52 -07:00
Jane Wang f76790557b [SPARK-4131] Support "Writing data into the filesystem from queries"
## What changes were proposed in this pull request?

This PR implements the sql feature:
INSERT OVERWRITE [LOCAL] DIRECTORY directory1
  [ROW FORMAT row_format] [STORED AS file_format]
  SELECT ... FROM ...

## How was this patch tested?
Added new unittests and also pulled the code to fb-spark so that we could test writing to hdfs directory.

Author: Jane Wang <janewang@fb.com>

Closes #18975 from janewangfb/port_local_directory.
2017-09-09 11:48:34 -07:00
Liang-Chi Hsieh 6b45d7e941 [SPARK-21954][SQL] JacksonUtils should verify MapType's value type instead of key type
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

Added tests.

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

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

The condition in `Optimizer.isPlanIntegral` is wrong. We should always return `true` if not in test mode.

## How was this patch tested?

Manually test.

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

Closes #19161 from viirya/SPARK-21726-followup.
2017-09-08 20:21:37 +09:00
Liang-Chi Hsieh 6e37524a1f [SPARK-21726][SQL] Check for structural integrity of the plan in Optimzer in test mode.
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

Added tests.

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

Closes #18956 from viirya/SPARK-21726.
2017-09-07 23:12:18 -07:00
Dongjoon Hyun e00f1a1da1 [SPARK-13656][SQL] Delete spark.sql.parquet.cacheMetadata from SQLConf and docs
## What changes were proposed in this pull request?

Since [SPARK-15639](https://github.com/apache/spark/pull/13701), `spark.sql.parquet.cacheMetadata` and `PARQUET_CACHE_METADATA` is not used. This PR removes from SQLConf and docs.

## How was this patch tested?

Pass the existing Jenkins.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19129 from dongjoon-hyun/SPARK-13656.
2017-09-07 16:26:56 -07:00
Liang-Chi Hsieh ce7293c150 [SPARK-21835][SQL][FOLLOW-UP] RewritePredicateSubquery should not produce unresolved query plans
## What changes were proposed in this pull request?

This is a follow-up of #19050 to deal with `ExistenceJoin` case.

## How was this patch tested?

Added test.

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

Closes #19151 from viirya/SPARK-21835-followup.
2017-09-06 22:15:25 -07:00
Jose Torres acdf45fb52 [SPARK-21765] Check that optimization doesn't affect isStreaming bit.
## What changes were proposed in this pull request?

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

## How was this patch tested?

new and existing unit tests

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

Closes #19056 from joseph-torres/SPARK-21765-followup.
2017-09-06 11:19:46 -07:00
Liang-Chi Hsieh f2e22aebfe [SPARK-21835][SQL] RewritePredicateSubquery should not produce unresolved query plans
## What changes were proposed in this pull request?

Correlated predicate subqueries are rewritten into `Join` by the rule `RewritePredicateSubquery`  during optimization.

It is possibly that the two sides of the `Join` have conflicting attributes. The query plans produced by `RewritePredicateSubquery` become unresolved and break structural integrity.

We should check if there are conflicting attributes in the `Join` and de-duplicate them by adding a `Project`.

## How was this patch tested?

Added tests.

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

Closes #19050 from viirya/SPARK-21835.
2017-09-06 07:42:19 -07:00
Xingbo Jiang fd60d4fa6c [SPARK-21652][SQL] Fix rule confliction between InferFiltersFromConstraints and ConstantPropagation
## What changes were proposed in this pull request?

For the given example below, the predicate added by `InferFiltersFromConstraints` is folded by `ConstantPropagation` later, this leads to unconverged optimize iteration:
```
Seq((1, 1)).toDF("col1", "col2").createOrReplaceTempView("t1")
Seq(1, 2).toDF("col").createOrReplaceTempView("t2")
sql("SELECT * FROM t1, t2 WHERE t1.col1 = 1 AND 1 = t1.col2 AND t1.col1 = t2.col AND t1.col2 = t2.col")
```

We can fix this by adjusting the indent of the optimize rules.

## How was this patch tested?

Add test case that would have failed in `SQLQuerySuite`.

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

Closes #19099 from jiangxb1987/unconverge-optimization.
2017-09-05 13:12:39 -07:00
gatorsmile 2974406d17 [SPARK-21845][SQL][TEST-MAVEN] Make codegen fallback of expressions configurable
## What changes were proposed in this pull request?
We should make codegen fallback of expressions configurable. So far, it is always on. We might hide it when our codegen have compilation bugs. Thus, we should also disable the codegen fallback when running test cases.

## How was this patch tested?
Added test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19119 from gatorsmile/fallbackCodegen.
2017-09-05 09:04:03 -07:00
Liang-Chi Hsieh 9f30d92803 [SPARK-21654][SQL] Complement SQL predicates expression description
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

Existing tests. And added predicate test.

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

Closes #18869 from viirya/SPARK-21654.
2017-09-03 21:55:18 -07:00
gatorsmile acb7fed237 [SPARK-21891][SQL] Add TBLPROPERTIES to DDL statement: CREATE TABLE USING
## What changes were proposed in this pull request?
Add `TBLPROPERTIES` to the DDL statement `CREATE TABLE USING`.

After this change, the DDL becomes
```
CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db_name.]table_name
USING table_provider
[OPTIONS table_property_list]
[PARTITIONED BY (col_name, col_name, ...)]
[CLUSTERED BY (col_name, col_name, ...)
 [SORTED BY (col_name [ASC|DESC], ...)]
 INTO num_buckets BUCKETS
]
[LOCATION path]
[COMMENT table_comment]
[TBLPROPERTIES (property_name=property_value, ...)]
[[AS] select_statement];
```

## How was this patch tested?
Add a few tests

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19100 from gatorsmile/addTablePropsToCreateTableUsing.
2017-09-02 14:53:41 -07:00
Sean Owen 12ab7f7e89 [SPARK-14280][BUILD][WIP] Update change-version.sh and pom.xml to add Scala 2.12 profiles and enable 2.12 compilation
…build; fix some things that will be warnings or errors in 2.12; restore Scala 2.12 profile infrastructure

## What changes were proposed in this pull request?

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

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

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

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

This change does _not_ fully enable a Scala 2.12 build:

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

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

## How was this patch tested?

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

Author: Sean Owen <sowen@cloudera.com>

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

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

## How was this patch tested?

Updated unit tests to cover structs and arrays.

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

Closes #18818 from aray/SPARK-21110.
2017-08-31 15:08:03 -07:00
gatorsmile 7ce1108286 [SPARK-17107][SQL][FOLLOW-UP] Remove redundant pushdown rule for Union
## What changes were proposed in this pull request?
Also remove useless function `partitionByDeterministic` after the changes of https://github.com/apache/spark/pull/14687

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19097 from gatorsmile/followupSPARK-17107.
2017-08-31 14:15:34 -07:00
Dongjoon Hyun d8f4540863 [SPARK-21839][SQL] Support SQL config for ORC compression
## What changes were proposed in this pull request?

This PR aims to support `spark.sql.orc.compression.codec` like Parquet's `spark.sql.parquet.compression.codec`. Users can use SQLConf to control ORC compression, too.

## How was this patch tested?

Pass the Jenkins with new and updated test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19055 from dongjoon-hyun/SPARK-21839.
2017-08-31 08:16:58 +09:00
gatorsmile 32d6d9d720 Revert "[SPARK-21845][SQL] Make codegen fallback of expressions configurable"
This reverts commit 3d0e174244.
2017-08-30 09:08:40 -07:00
gatorsmile 3d0e174244 [SPARK-21845][SQL] Make codegen fallback of expressions configurable
## What changes were proposed in this pull request?
We should make codegen fallback of expressions configurable. So far, it is always on. We might hide it when our codegen have compilation bugs. Thus, we should also disable the codegen fallback when running test cases.

## How was this patch tested?
Added test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19062 from gatorsmile/fallbackCodegen.
2017-08-29 20:59:01 -07:00
Wenchen Fan 6327ea570b [SPARK-21255][SQL] simplify encoder for java enum
## What changes were proposed in this pull request?

This is a follow-up for https://github.com/apache/spark/pull/18488, to simplify the code.

The major change is, we should map java enum to string type, instead of a struct type with a single string field.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19066 from cloud-fan/fix.
2017-08-29 09:15:59 -07:00
Wang Gengliang 8fcbda9c93 [SPARK-21848][SQL] Add trait UserDefinedExpression to identify user-defined functions
## What changes were proposed in this pull request?

Add trait UserDefinedExpression to identify user-defined functions.
UDF can be expensive. In optimizer we may need to avoid executing UDF multiple times.
E.g.
```scala
table.select(UDF as 'a).select('a, ('a + 1) as 'b)
```
If UDF is expensive in this case, optimizer should not collapse the project to
```scala
table.select(UDF as 'a, (UDF+1) as 'b)
```

Currently UDF classes like PythonUDF, HiveGenericUDF are not defined in catalyst.
This PR is to add a new trait to make it easier to identify user-defined functions.

## How was this patch tested?

Unit test

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #19064 from gengliangwang/UDFType.
2017-08-29 09:08:59 -07:00
vinodkc 51620e288b [SPARK-21756][SQL] Add JSON option to allow unquoted control characters
## What changes were proposed in this pull request?

This patch adds allowUnquotedControlChars option in JSON data source to allow JSON Strings to contain unquoted control characters (ASCII characters with value less than 32, including tab and line feed characters)

## How was this patch tested?
Add new test cases

Author: vinodkc <vinod.kc.in@gmail.com>

Closes #19008 from vinodkc/br_fix_SPARK-21756.
2017-08-25 10:18:03 -07:00
mike 7d16776d28 [SPARK-21255][SQL][WIP] Fixed NPE when creating encoder for enum
## What changes were proposed in this pull request?

Fixed NPE when creating encoder for enum.

When you try to create an encoder for Enum type (or bean with enum property) via Encoders.bean(...), it fails with NullPointerException at TypeToken:495.
I did a little research and it turns out, that in JavaTypeInference following code
```
  def getJavaBeanReadableProperties(beanClass: Class[_]): Array[PropertyDescriptor] = {
    val beanInfo = Introspector.getBeanInfo(beanClass)
    beanInfo.getPropertyDescriptors.filterNot(_.getName == "class")
      .filter(_.getReadMethod != null)
  }
```
filters out properties named "class", because we wouldn't want to serialize that. But enum types have another property of type Class named "declaringClass", which we are trying to inspect recursively. Eventually we try to inspect ClassLoader class, which has property "defaultAssertionStatus" with no read method, which leads to NPE at TypeToken:495.

I added property name "declaringClass" to filtering to resolve this.

## How was this patch tested?
Unit test in JavaDatasetSuite which creates an encoder for enum

Author: mike <mike0sv@gmail.com>
Author: Mikhail Sveshnikov <mike0sv@gmail.com>

Closes #18488 from mike0sv/enum-support.
2017-08-25 07:22:34 +01:00
Herman van Hovell 05af2de0fd [SPARK-21830][SQL] Bump ANTLR version and fix a few issues.
## What changes were proposed in this pull request?
This PR bumps the ANTLR version to 4.7, and fixes a number of small parser related issues uncovered by the bump.

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

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

## How was this patch tested?
Existing tests.

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

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

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

For a correlated IN query looks like:

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

The query plan might look like:

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

After `PullupCorrelatedPredicates`, it produces query plan like:

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

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

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

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

## How was this patch tested?

Added test.

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

Closes #18968 from viirya/SPARK-21759.
2017-08-24 21:46:58 +08:00
Takuya UESHIN 9e33954ddf [SPARK-21745][SQL] Refactor ColumnVector hierarchy to make ColumnVector read-only and to introduce WritableColumnVector.
## What changes were proposed in this pull request?

This is a refactoring of `ColumnVector` hierarchy and related classes.

1. make `ColumnVector` read-only
2. introduce `WritableColumnVector` with write interface
3. remove `ReadOnlyColumnVector`

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #18958 from ueshin/issues/SPARK-21745.
2017-08-24 21:13:44 +08:00
Jen-Ming Chung 95713eb4f2 [SPARK-21804][SQL] json_tuple returns null values within repeated columns except the first one
## What changes were proposed in this pull request?

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

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

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

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

## How was this patch tested?

Added test in JsonExpressionsSuite.

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

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

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

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

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

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

(Please fill in changes proposed in this fix)

## How was this patch tested?

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

Run new added test.

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

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

Closes #19022 from eatoncys/getAliasedConstraints.
2017-08-23 20:35:08 -07:00
Takeshi Yamamuro 6942aeeb0a [SPARK-21603][SQL][FOLLOW-UP] Change the default value of maxLinesPerFunction into 4000
## What changes were proposed in this pull request?
This pr changed the default value of `maxLinesPerFunction` into `4000`. In #18810, we had this new option to disable code generation for too long functions and I found this option only affected `Q17` and `Q66` in TPC-DS. But, `Q66` had some performance regression:

```
Q17 w/o #18810, 3224ms --> q17 w/#18810, 2627ms (improvement)
Q66 w/o #18810, 1712ms --> q66 w/#18810, 3032ms (regression)
```

To keep the previous performance in TPC-DS, we better set higher value at `maxLinesPerFunction` by default.

## How was this patch tested?
Existing tests.

Author: Takeshi Yamamuro <yamamuro@apache.org>

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

## How was this patch tested?

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

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

Closes #18973 from joseph-torres/SPARK-21765.
2017-08-22 19:07:43 -07:00
gatorsmile 43d71d9659 [SPARK-21499][SQL] Support creating persistent function for Spark UDAF(UserDefinedAggregateFunction)
## What changes were proposed in this pull request?
This PR is to enable users to create persistent Scala UDAF (that extends UserDefinedAggregateFunction).

```SQL
CREATE FUNCTION myDoubleAvg AS 'test.org.apache.spark.sql.MyDoubleAvg'
```

Before this PR, Spark UDAF only can be registered through the API `spark.udf.register(...)`

## How was this patch tested?
Added test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18700 from gatorsmile/javaUDFinScala.
2017-08-22 13:01:35 -07:00
Wenchen Fan 7880909c45 [SPARK-21743][SQL][FOLLOW-UP] top-most limit should not cause memory leak
## What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/18955 , to fix a bug that we break whole stage codegen for `Limit`.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18993 from cloud-fan/bug.
2017-08-18 11:19:22 -07:00
Masha Basmanova 23ea898080 [SPARK-21213][SQL] Support collecting partition-level statistics: rowCount and sizeInBytes
## What changes were proposed in this pull request?

Added support for ANALYZE TABLE [db_name].tablename PARTITION (partcol1[=val1], partcol2[=val2], ...) COMPUTE STATISTICS [NOSCAN] SQL command to calculate total number of rows and size in bytes for a subset of partitions. Calculated statistics are stored in Hive Metastore as user-defined properties attached to partition objects. Property names are the same as the ones used to store table-level statistics: spark.sql.statistics.totalSize and spark.sql.statistics.numRows.

When partition specification contains all partition columns with values, the command collects statistics for a single partition that matches the specification. When some partition columns are missing or listed without their values, the command collects statistics for all partitions which match a subset of partition column values specified.

For example, table t has 4 partitions with the following specs:

* Partition1: (ds='2008-04-08', hr=11)
* Partition2: (ds='2008-04-08', hr=12)
* Partition3: (ds='2008-04-09', hr=11)
* Partition4: (ds='2008-04-09', hr=12)

'ANALYZE TABLE t PARTITION (ds='2008-04-09', hr=11)' command will collect statistics only for partition 3.

'ANALYZE TABLE t PARTITION (ds='2008-04-09')' command will collect statistics for partitions 3 and 4.

'ANALYZE TABLE t PARTITION (ds, hr)' command will collect statistics for all four partitions.

When the optional parameter NOSCAN is specified, the command doesn't count number of rows and only gathers size in bytes.

The statistics gathered by ANALYZE TABLE command can be fetched using DESC EXTENDED [db_name.]tablename PARTITION command.

## How was this patch tested?

Added tests.

Author: Masha Basmanova <mbasmanova@fb.com>

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

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

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

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

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

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

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

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

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

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

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

    benchmark.run()

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

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

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

Closes #18810 from eatoncys/codegen.
2017-08-16 09:12:20 -07:00
WeichenXu 07549b20a3 [SPARK-19634][ML] Multivariate summarizer - dataframes API
## What changes were proposed in this pull request?

This patch adds the DataFrames API to the multivariate summarizer (mean, variance, etc.). In addition to all the features of MultivariateOnlineSummarizer, it also allows the user to select a subset of the metrics.

## How was this patch tested?

Testcases added.

## Performance
Resolve several performance issues in #17419, further optimization pending on SQL team's work. One of the SQL layer performance issue related to these feature has been resolved in #18712, thanks liancheng and cloud-fan

### Performance data

(test on my laptop, use 2 partitions. tries out = 20, warm up = 10)

The unit of test results is records/milliseconds (higher is better)

Vector size/records number | 1/10000000 | 10/1000000 | 100/1000000 | 1000/100000 | 10000/10000
----|------|----|---|----|----
Dataframe | 15149  | 7441 | 2118 | 224 | 21
RDD from Dataframe | 4992  | 4440 | 2328 | 320 | 33
raw RDD | 53931  | 20683 | 3966 | 528 | 53

Author: WeichenXu <WeichenXu123@outlook.com>

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

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

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

Author: Marcelo Vanzin <vanzin@cloudera.com>

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

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

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

## How was this patch tested?

regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18907 from cloud-fan/bug.
2017-08-15 09:04:56 -07:00
hyukjinkwon 0422ce06df [SPARK-21724][SQL][DOC] Adds since information in the documentation of date functions
## What changes were proposed in this pull request?

This PR adds `since` annotation in documentation so that this can be rendered as below:

<img width="290" alt="2017-08-14 6 54 26" src="https://user-images.githubusercontent.com/6477701/29267050-034c1f64-8122-11e7-862b-7dfc38e292bf.png">

## How was this patch tested?

Manually checked the documentation by `cd sql && ./create-docs.sh`.
Also, Jenkins tests are required.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18939 from HyukjinKwon/add-sinces-date-functions.
2017-08-14 23:44:25 -07:00
aokolnychyi 5596ce83c4 [MINOR][SQL] Additional test case for CheckCartesianProducts rule
## What changes were proposed in this pull request?

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

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

Closes #18909 from aokolnychyi/check-cartesian-join-tests.
2017-08-13 21:33:16 -07:00
Tejas Patil 94439997d5 [SPARK-21595] Separate thresholds for buffering and spilling in ExternalAppendOnlyUnsafeRowArray
## What changes were proposed in this pull request?

[SPARK-21595](https://issues.apache.org/jira/browse/SPARK-21595) reported that there is excessive spilling to disk due to default spill threshold for `ExternalAppendOnlyUnsafeRowArray` being quite small for WINDOW operator. Old behaviour of WINDOW operator (pre https://github.com/apache/spark/pull/16909) would hold data in an array for first 4096 records post which it would switch to `UnsafeExternalSorter` and start spilling to disk after reaching `spark.shuffle.spill.numElementsForceSpillThreshold` (or earlier if there was paucity of memory due to excessive consumers).

Currently the (switch from in-memory to `UnsafeExternalSorter`) and (`UnsafeExternalSorter` spilling to disk) for `ExternalAppendOnlyUnsafeRowArray` is controlled by a single threshold. This PR aims to separate that to have more granular control.

## How was this patch tested?

Added unit tests

Author: Tejas Patil <tejasp@fb.com>

Closes #18843 from tejasapatil/SPARK-21595.
2017-08-11 22:01:00 +02:00
Reynold Xin 584c7f1437 [SPARK-21699][SQL] Remove unused getTableOption in ExternalCatalog
## What changes were proposed in this pull request?
This patch removes the unused SessionCatalog.getTableMetadataOption and ExternalCatalog. getTableOption.

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

Author: Reynold Xin <rxin@databricks.com>

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

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

## How was this patch tested?
unit tests

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

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

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18709 from gatorsmile/addVersion.
2017-08-09 08:46:25 -07:00
Xingbo Jiang 031910b0ec [SPARK-21608][SPARK-9221][SQL] Window rangeBetween() API should allow literal boundary
## What changes were proposed in this pull request?

Window rangeBetween() API should allow literal boundary, that means, the window range frame can calculate frame of double/date/timestamp.

Example of the use case can be:
```
SELECT
	val_timestamp,
	cate,
	avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_timestamp RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING)
FROM testData
```

This PR refactors the Window `rangeBetween` and `rowsBetween` API, while the legacy user code should still be valid.

## How was this patch tested?

Add new test cases both in `DataFrameWindowFunctionsSuite` and in `window.sql`.

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

Closes #18814 from jiangxb1987/literal-boundary.
2017-08-09 13:23:49 +08:00
Liang-Chi Hsieh ee1304199b [SPARK-21567][SQL] Dataset should work with type alias
## What changes were proposed in this pull request?

If we create a type alias for a type workable with Dataset, the type alias doesn't work with Dataset.

A reproducible case looks like:

    object C {
      type TwoInt = (Int, Int)
      def tupleTypeAlias: TwoInt = (1, 1)
    }

    Seq(1).toDS().map(_ => ("", C.tupleTypeAlias))

It throws an exception like:

    type T1 is not a class
    scala.ScalaReflectionException: type T1 is not a class
      at scala.reflect.api.Symbols$SymbolApi$class.asClass(Symbols.scala:275)
      ...

This patch accesses the dealias of type in many places in `ScalaReflection` to fix it.

## How was this patch tested?

Added test case.

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

Closes #18813 from viirya/SPARK-21567.
2017-08-08 16:12:41 +08:00
zhoukang 8b69b17f3f [SPARK-21544][DEPLOY][TEST-MAVEN] Tests jar of some module should not upload twice
## What changes were proposed in this pull request?

**For moudle below:**
common/network-common
streaming
sql/core
sql/catalyst
**tests.jar will install or deploy twice.Like:**
`[DEBUG] Installing org.apache.spark:spark-streaming_2.11/maven-metadata.xml to /home/mi/.m2/repository/org/apache/spark/spark-streaming_2.11/maven-metadata-local.xml
[INFO] Installing /home/mi/Work/Spark/scala2.11/spark/streaming/target/spark-streaming_2.11-2.1.0-mdh2.1.0.1-SNAPSHOT-tests.jar to /home/mi/.m2/repository/org/apache/spark/spark-streaming_2.11/2.1.0-mdh2.1.0.1-SNAPSHOT/spark-streaming_2.11-2.1.0-mdh2.1.0.1-SNAPSHOT-tests.jar
[DEBUG] Skipped re-installing /home/mi/Work/Spark/scala2.11/spark/streaming/target/spark-streaming_2.11-2.1.0-mdh2.1.0.1-SNAPSHOT-tests.jar to /home/mi/.m2/repository/org/apache/spark/spark-streaming_2.11/2.1.0-mdh2.1.0.1-SNAPSHOT/spark-streaming_2.11-2.1.0-mdh2.1.0.1-SNAPSHOT-tests.jar, seems unchanged`
**The reason is below:**
`[DEBUG]   (f) artifact = org.apache.spark:spark-streaming_2.11🫙2.1.0-mdh2.1.0.1-SNAPSHOT
[DEBUG]   (f) attachedArtifacts = [org.apache.spark:spark-streaming_2.11:test-jar:tests:2.1.0-mdh2.1.0.1-SNAPSHOT, org.apache.spark:spark-streaming_2.11🫙tests:2.1.0-mdh2.1.0.1-SNAPSHOT, org.apache.spark:spark
-streaming_2.11:java-source:sources:2.1.0-mdh2.1.0.1-SNAPSHOT, org.apache.spark:spark-streaming_2.11:java-source:test-sources:2.1.0-mdh2.1.0.1-SNAPSHOT, org.apache.spark:spark-streaming_2.11:javadoc:javadoc:2.1.0
-mdh2.1.0.1-SNAPSHOT]`

when executing 'mvn deploy' to nexus during release.I will fail since release nexus can not be overrided.

## How was this patch tested?
Execute 'mvn clean install -Pyarn -Phadoop-2.6 -Phadoop-provided -DskipTests'

Author: zhoukang <zhoukang199191@gmail.com>

Closes #18745 from caneGuy/zhoukang/fix-installtwice.
2017-08-07 12:51:39 +01:00
BartekH 438c381584 Add "full_outer" name to join types
I have discovered that "full_outer" name option is working in Spark 2.0, but it is not printed in exception. Please verify.

## What changes were proposed in this pull request?

(Please fill in changes proposed in this fix)

## How was this patch tested?

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

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

Author: BartekH <bartekhamielec@gmail.com>

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

## How was this patch tested?
Existing tests

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #18857 from maropu/SPARK-20963-FOLLOWUP.
2017-08-06 10:14:45 -07:00
vinodkc 1ba967b25e [SPARK-21588][SQL] SQLContext.getConf(key, null) should return null
## What changes were proposed in this pull request?

In SQLContext.get(key,null) for a key that is not defined in the conf, and doesn't have a default value defined, throws a NPE. Int happens only when conf has a value converter

Added null check on defaultValue inside SQLConf.getConfString to avoid calling entry.valueConverter(defaultValue)

## How was this patch tested?
Added unit test

Author: vinodkc <vinod.kc.in@gmail.com>

Closes #18852 from vinodkc/br_Fix_SPARK-21588.
2017-08-05 23:04:39 -07:00
Takeshi Yamamuro 990efad1c6 [SPARK-20963][SQL] Support column aliases for join relations in FROM clause
## What changes were proposed in this pull request?
This pr added parsing rules to support column aliases for join relations in FROM clause.
This pr is a sub-task of #18079.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #18772 from maropu/SPARK-20963-2.
2017-08-05 20:35:54 -07:00
hyukjinkwon ba327ee54c [SPARK-21485][FOLLOWUP][SQL][DOCS] Describes examples and arguments separately, and note/since in SQL built-in function documentation
## What changes were proposed in this pull request?

This PR proposes to separate `extended` into `examples` and `arguments` internally so that both can be separately documented and add `since` and `note` for additional information.

For `since`, it looks users sometimes get confused by, up to my knowledge, missing version information. For example, see https://www.mail-archive.com/userspark.apache.org/msg64798.html

For few good examples to check the built documentation, please see both:
`from_json` - https://spark-test.github.io/sparksqldoc/#from_json
`like` - https://spark-test.github.io/sparksqldoc/#like

For `DESCRIBE FUNCTION`, `note` and `since` are added as below:

```
> DESCRIBE FUNCTION EXTENDED rlike;
...
Extended Usage:
    Arguments:
      ...

    Examples:
      ...

    Note:
      Use LIKE to match with simple string pattern
```

```
> DESCRIBE FUNCTION EXTENDED to_json;
...
    Examples:
      ...

    Since: 2.2.0
```

For the complete documentation, see https://spark-test.github.io/sparksqldoc/

## How was this patch tested?

Manual tests and existing tests. Please see https://spark-test.github.io/sparksqldoc

Jenkins tests are needed to double check

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18749 from HyukjinKwon/followup-sql-doc-gen.
2017-08-05 10:10:56 -07:00
liuxian 894d5a453a [SPARK-21580][SQL] Integers in aggregation expressions are wrongly taken as group-by ordinal
## What changes were proposed in this pull request?

create temporary view data as select * from values
(1, 1),
(1, 2),
(2, 1),
(2, 2),
(3, 1),
(3, 2)
as data(a, b);

`select 3, 4, sum(b) from data group by 1, 2;`
`select 3 as c, 4 as d, sum(b) from data group by c, d;`
When running these two cases, the following exception occurred:
`Error in query: GROUP BY position 4 is not in select list (valid range is [1, 3]); line 1 pos 10`

The cause of this failure:
If an aggregateExpression is integer, after replaced with this aggregateExpression, the
groupExpression still considered as an ordinal.

The solution:
This bug is due to re-entrance of an analyzed plan. We can solve it by using `resolveOperators` in `SubstituteUnresolvedOrdinals`.

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

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

Closes #18779 from 10110346/groupby.
2017-08-04 22:55:06 -07:00
Reynold Xin 5ad1796b9f [SPARK-21634][SQL] Change OneRowRelation from a case object to case class
## What changes were proposed in this pull request?
OneRowRelation is the only plan that is a case object, which causes some issues with makeCopy using a 0-arg constructor. This patch changes it from a case object to a case class.

This blocks SPARK-21619.

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

Author: Reynold Xin <rxin@databricks.com>

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

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

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

## How was this patch tested?
unit tests

Author: Yuming Wang <wgyumg@gmail.com>

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

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

## How was this patch tested?

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

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

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

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

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

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

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

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

After this been merged, we can close #16818 .

## How was this patch tested?

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

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

Closes #18540 from jiangxb1987/rangeFrame.
2017-07-29 10:11:31 -07:00
Liang-Chi Hsieh 9c8109ef41 [SPARK-21555][SQL] RuntimeReplaceable should be compared semantically by its canonicalized child
## What changes were proposed in this pull request?

When there are aliases (these aliases were added for nested fields) as parameters in `RuntimeReplaceable`, as they are not in the children expression, those aliases can't be cleaned up in analyzer rule `CleanupAliases`.

An expression `nvl(foo.foo1, "value")` can be resolved to two semantically different expressions in a group by query because they contain different aliases.

Because those aliases are not children of `RuntimeReplaceable` which is an `UnaryExpression`. So we can't trim the aliases out by simple transforming the expressions in `CleanupAliases`.

If we want to replace the non-children aliases in `RuntimeReplaceable`, we need to add more codes to `RuntimeReplaceable` and modify all expressions of `RuntimeReplaceable`. It makes the interface ugly IMO.

Consider those aliases will be replaced later at optimization and so they're no harm, this patch chooses to simply override `canonicalized` of `RuntimeReplaceable`.

One concern is about `CleanupAliases`. Because it actually cannot clean up ALL aliases inside a plan. To make caller of this rule notice that, this patch adds a comment to `CleanupAliases`.

## How was this patch tested?

Added test.

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

Closes #18761 from viirya/SPARK-21555.
2017-07-29 10:02:56 -07:00
Wenchen Fan 9f5647d62e [SPARK-21319][SQL] Fix memory leak in sorter
## What changes were proposed in this pull request?

`UnsafeExternalSorter.recordComparator` can be either `KVComparator` or `RowComparator`, and both of them will keep the reference to the input rows they compared last time.

After sorting, we return the sorted iterator to upstream operators. However, the upstream operators may take a while to consume up the sorted iterator, and `UnsafeExternalSorter` is registered to `TaskContext` at [here](https://github.com/apache/spark/blob/v2.2.0/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java#L159-L161), which means we will keep the `UnsafeExternalSorter` instance and keep the last compared input rows in memory until the sorted iterator is consumed up.

Things get worse if we sort within partitions of a dataset and coalesce all partitions into one, as we will keep a lot of input rows in memory and the time to consume up all the sorted iterators is long.

This PR takes over https://github.com/apache/spark/pull/18543 , the idea is that, we do not keep the record comparator instance in `UnsafeExternalSorter`, but a generator of record comparator.

close #18543

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18679 from cloud-fan/memory-leak.
2017-07-27 22:56:26 +08:00
Kazuaki Ishizaki ebbe589d12 [SPARK-21271][SQL] Ensure Unsafe.sizeInBytes is a multiple of 8
## What changes were proposed in this pull request?

This PR ensures that `Unsafe.sizeInBytes` must be a multiple of 8. It it is not satisfied. `Unsafe.hashCode` causes the assertion violation.

## How was this patch tested?

Will add test cases

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

Closes #18503 from kiszk/SPARK-21271.
2017-07-27 15:27:24 +08:00
gatorsmile ebc24a9b7f [SPARK-20586][SQL] Add deterministic to ScalaUDF
### What changes were proposed in this pull request?
Like [Hive UDFType](https://hive.apache.org/javadocs/r2.0.1/api/org/apache/hadoop/hive/ql/udf/UDFType.html), we should allow users to add the extra flags for ScalaUDF and JavaUDF too. _stateful_/_impliesOrder_ are not applicable to our Scala UDF. Thus, we only add the following two flags.

- deterministic: Certain optimizations should not be applied if UDF is not deterministic. Deterministic UDF returns same result each time it is invoked with a particular input. This determinism just needs to hold within the context of a query.

When the deterministic flag is not correctly set, the results could be wrong.

For ScalaUDF in Dataset APIs, users can call the following extra APIs for `UserDefinedFunction` to make the corresponding changes.
- `nonDeterministic`: Updates UserDefinedFunction to non-deterministic.

Also fixed the Java UDF name loss issue.

Will submit a separate PR for `distinctLike`  for UDAF

### How was this patch tested?
Added test cases for both ScalaUDF

Author: gatorsmile <gatorsmile@gmail.com>
Author: Wenchen Fan <cloud0fan@gmail.com>

Closes #17848 from gatorsmile/udfRegister.
2017-07-25 17:19:44 -07:00
pj.fanning 2a53fbfce7 [SPARK-20871][SQL] limit logging of Janino code
## What changes were proposed in this pull request?

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

## How was this patch tested?

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

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

Closes #18658 from pjfanning/SPARK-20871.
2017-07-23 10:38:03 -07:00
Wenchen Fan 3ac6093086 [SPARK-10063] Follow-up: remove dead code related to an old output committer
## What changes were proposed in this pull request?

DirectParquetOutputCommitter was removed from Spark as it was deemed unsafe to use. We however still have some code to generate warning. This patch removes those code as well.

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

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

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

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

Author: gatorsmile <gatorsmile@gmail.com>

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

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

## How was this patch tested?

new regression test.

Author: Wenchen Fan <wenchen@databricks.com>

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

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

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

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

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

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

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

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

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

The example above gives the following output:

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

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

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

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

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

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

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

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

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

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

## How was this patch tested?

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

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

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

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

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

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

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

## How was this patch tested?

Added a test suite in `OrderingSuite`.

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

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

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

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #17150 from srowen/SPARK-19810.
2017-07-13 17:06:24 +08:00
liuxian aaad34dc2f [SPARK-21007][SQL] Add SQL function - RIGHT && LEFT
## What changes were proposed in this pull request?
 Add  SQL function - RIGHT && LEFT, same as MySQL:
https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_left
https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_right

## How was this patch tested?
unit test

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

Closes #18228 from 10110346/lx-wip-0607.
2017-07-12 18:51:19 +08:00
Jane Wang 2cbfc975ba [SPARK-12139][SQL] REGEX Column Specification
## What changes were proposed in this pull request?
Hive interprets regular expression, e.g., `(a)?+.+` in query specification. This PR enables spark to support this feature when hive.support.quoted.identifiers is set to true.

## How was this patch tested?

- Add unittests in SQLQuerySuite.scala
- Run spark-shell tested the original failed query:
scala> hc.sql("SELECT `(a|b)?+.+` from test1").collect.foreach(println)

Author: Jane Wang <janewang@fb.com>

Closes #18023 from janewangfb/support_select_regex.
2017-07-11 22:00:36 -07:00
Bryan Cutler d03aebbe65 [SPARK-13534][PYSPARK] Using Apache Arrow to increase performance of DataFrame.toPandas
## What changes were proposed in this pull request?
Integrate Apache Arrow with Spark to increase performance of `DataFrame.toPandas`.  This has been done by using Arrow to convert data partitions on the executor JVM to Arrow payload byte arrays where they are then served to the Python process.  The Python DataFrame can then collect the Arrow payloads where they are combined and converted to a Pandas DataFrame.  Data types except complex, date, timestamp, and decimal  are currently supported, otherwise an `UnsupportedOperation` exception is thrown.

Additions to Spark include a Scala package private method `Dataset.toArrowPayload` that will convert data partitions in the executor JVM to `ArrowPayload`s as byte arrays so they can be easily served.  A package private class/object `ArrowConverters` that provide data type mappings and conversion routines.  In Python, a private method `DataFrame._collectAsArrow` is added to collect Arrow payloads and a SQLConf "spark.sql.execution.arrow.enable" can be used in `toPandas()` to enable using Arrow (uses the old conversion by default).

## How was this patch tested?
Added a new test suite `ArrowConvertersSuite` that will run tests on conversion of Datasets to Arrow payloads for supported types.  The suite will generate a Dataset and matching Arrow JSON data, then the dataset is converted to an Arrow payload and finally validated against the JSON data.  This will ensure that the schema and data has been converted correctly.

Added PySpark tests to verify the `toPandas` method is producing equal DataFrames with and without pyarrow.  A roundtrip test to ensure the pandas DataFrame produced by pyspark is equal to a one made directly with pandas.

Author: Bryan Cutler <cutlerb@gmail.com>
Author: Li Jin <ice.xelloss@gmail.com>
Author: Li Jin <li.jin@twosigma.com>
Author: Wes McKinney <wes.mckinney@twosigma.com>

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

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

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

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

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

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

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

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

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

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

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17758 from maropu/SPARK-20460.
2017-07-10 15:58:34 +08:00
Wenchen Fan 680b33f166 [SPARK-18016][SQL][FOLLOWUP] merge declareAddedFunctions, initNestedClasses and declareNestedClasses
## What changes were proposed in this pull request?

These 3 methods have to be used together, so it makes more sense to merge them into one method and then the caller side only need to call one method.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

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

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

Author: Xiao Li <gatorsmile@gmail.com>

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

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

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

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

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

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

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

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

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18559 from cloud-fan/sub-query.
2017-07-07 20:04:30 +08:00
Wang Gengliang bf66335aca [SPARK-21323][SQL] Rename plans.logical.statsEstimation.Range to ValueInterval
## What changes were proposed in this pull request?

Rename org.apache.spark.sql.catalyst.plans.logical.statsEstimation.Range to ValueInterval.
The current naming is identical to logical operator "range".
Refactoring it to ValueInterval is more accurate.

## How was this patch tested?

unit test

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

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #18549 from gengliangwang/ValueInterval.
2017-07-06 13:58:27 -07:00
Liang-Chi Hsieh 48e44b24a7 [SPARK-21204][SQL] Add support for Scala Set collection types in serialization
## What changes were proposed in this pull request?

Currently we can't produce a `Dataset` containing `Set` in SparkSQL. This PR tries to support serialization/deserialization of `Set`.

Because there's no corresponding internal data type in SparkSQL for a `Set`, the most proper choice for serializing a set should be an array.

## How was this patch tested?

Added unit tests.

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

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

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

Author: Bogdan Raducanu <bogdan@databricks.com>

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

Add missing test cases back and revise code style

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

## How was this patch tested?

Unit test

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

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #18548 from gengliangwang/stat_propagation_revise.
2017-07-06 19:12:15 +08:00
Sumedh Wale 14a3bb3a00 [SPARK-21312][SQL] correct offsetInBytes in UnsafeRow.writeToStream
## What changes were proposed in this pull request?

Corrects offsetInBytes calculation in UnsafeRow.writeToStream. Known failures include writes to some DataSources that have own SparkPlan implementations and cause EXCHANGE in writes.

## How was this patch tested?

Extended UnsafeRowSuite.writeToStream to include an UnsafeRow over byte array having non-zero offset.

Author: Sumedh Wale <swale@snappydata.io>

Closes #18535 from sumwale/SPARK-21312.
2017-07-06 14:47:22 +08:00
gatorsmile 75b168fd30 [SPARK-21308][SQL] Remove SQLConf parameters from the optimizer
### What changes were proposed in this pull request?
This PR removes SQLConf parameters from the optimizer rules

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

Author: gatorsmile <gatorsmile@gmail.com>

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

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

Author: gatorsmile <gatorsmile@gmail.com>

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

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

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

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

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

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

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

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

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

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

Closes #13873 from ueshin/issues/SPARK-16167.
2017-07-05 20:32:47 +08:00
Takuya UESHIN a386432566 [SPARK-18623][SQL] Add returnNullable to StaticInvoke and modify it to handle properly.
## What changes were proposed in this pull request?

Add `returnNullable` to `StaticInvoke` the same as #15780 is trying to add to `Invoke` and modify to handle properly.

## How was this patch tested?

Existing tests.

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

Closes #16056 from ueshin/issues/SPARK-18623.
2017-07-05 14:25:26 +08:00
Wenchen Fan f2c3b1dd69 [SPARK-21304][SQL] remove unnecessary isNull variable for collection related encoder expressions
## What changes were proposed in this pull request?

For these collection-related encoder expressions, we don't need to create `isNull` variable if the loop element is not nullable.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18529 from cloud-fan/minor.
2017-07-05 14:17:26 +08:00
Takuya UESHIN ce10545d34 [SPARK-21300][SQL] ExternalMapToCatalyst should null-check map key prior to converting to internal value.
## What changes were proposed in this pull request?

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

## How was this patch tested?

Added a test and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #18524 from ueshin/issues/SPARK-21300.
2017-07-05 11:24:38 +08:00
gatorsmile de14086e1f [SPARK-21295][SQL] Use qualified names in error message for missing references
### What changes were proposed in this pull request?
It is strange to see the following error message. Actually, the column is from another table.
```
cannot resolve '`right.a`' given input columns: [a, c, d];
```

After the PR, the error message looks like
```
cannot resolve '`right.a`' given input columns: [left.a, right.c, right.d];
```

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

Author: gatorsmile <gatorsmile@gmail.com>

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

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

Author: gatorsmile <gatorsmile@gmail.com>

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

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

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18510 from cloud-fan/minor.
2017-07-03 10:51:44 -07:00
aokolnychyi 17bdc36ef1 [SPARK-21102][SQL] Refresh command is too aggressive in parsing
### Idea

This PR adds validation to REFRESH sql statements. Currently, users can specify whatever they want as resource path. For example, spark.sql("REFRESH ! $ !") will be executed without any exceptions.

### Implementation

I am not sure that my current implementation is the most optimal, so any feedback is appreciated. My first idea was to make the grammar as strict as possible. Unfortunately, there were some problems. I tried the approach below:

SqlBase.g4
```
...
    | REFRESH TABLE tableIdentifier                                    #refreshTable
    | REFRESH resourcePath                                             #refreshResource
...

resourcePath
    : STRING
    | (IDENTIFIER | number | nonReserved | '/' | '-')+ // other symbols can be added if needed
    ;
```
It is not flexible enough and requires to explicitly mention all possible symbols. Therefore, I came up with the current approach that is implemented in the code.

Let me know your opinion on which one is better.

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

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

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

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

Author: Reynold Xin <rxin@databricks.com>

Closes #18479 from rxin/stats-trait.
2017-06-30 21:10:23 -07:00
wangzhenhua 61b5df567e [SPARK-21127][SQL] Update statistics after data changing commands
## What changes were proposed in this pull request?

Update stats after the following data changing commands:

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

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

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

Closes #18334 from wzhfy/changeStatsForOperation.
2017-07-01 10:01:44 +08:00
Wenchen Fan 4eb41879ce [SPARK-17528][SQL] data should be copied properly before saving into InternalRow
## What changes were proposed in this pull request?

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

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

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

## How was this patch tested?

new regression tests

Author: Wenchen Fan <wenchen@databricks.com>

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

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

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

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

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

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

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

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

Invalidate spark's stats after data changing commands:

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

## How was this patch tested?

Added test cases.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #18449 from wzhfy/removeStats.
2017-06-29 11:32:29 +08:00
Wenchen Fan 25c2edf6f9 [SPARK-21229][SQL] remove QueryPlan.preCanonicalized
## What changes were proposed in this pull request?

`QueryPlan.preCanonicalized` is only overridden in a few places, and it does introduce an extra concept to `QueryPlan` which may confuse people.

This PR removes it and override `canonicalized` in these places

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18440 from cloud-fan/minor.
2017-06-29 11:21:50 +08:00
Wang Gengliang b72b8521d9 [SPARK-21222] Move elimination of Distinct clause from analyzer to optimizer
## What changes were proposed in this pull request?

Move elimination of Distinct clause from analyzer to optimizer

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

## How was this patch tested?

Unit test

gatorsmile cloud-fan

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

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #18429 from gengliangwang/distinct_opt.
2017-06-29 08:47:31 +08:00
Wenchen Fan 838effb98a Revert "[SPARK-13534][PYSPARK] Using Apache Arrow to increase performance of DataFrame.toPandas"
This reverts commit e44697606f.
2017-06-28 14:28:40 +08:00
Liang-Chi Hsieh fd8c931a30 [SPARK-19104][SQL] Lambda variables in ExternalMapToCatalyst should be global
## What changes were proposed in this pull request?

The issue happens in `ExternalMapToCatalyst`. For example, the following codes create `ExternalMapToCatalyst` to convert Scala Map to catalyst map format.

    val data = Seq.tabulate(10)(i => NestedData(1, Map("key" -> InnerData("name", i + 100))))
    val ds = spark.createDataset(data)

The `valueConverter` in `ExternalMapToCatalyst` looks like:

    if (isnull(lambdavariable(ExternalMapToCatalyst_value52, ExternalMapToCatalyst_value_isNull52, ObjectType(class org.apache.spark.sql.InnerData), true))) null else named_struct(name, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(lambdavariable(ExternalMapToCatalyst_value52, ExternalMapToCatalyst_value_isNull52, ObjectType(class org.apache.spark.sql.InnerData), true)).name, true), value, assertnotnull(lambdavariable(ExternalMapToCatalyst_value52, ExternalMapToCatalyst_value_isNull52, ObjectType(class org.apache.spark.sql.InnerData), true)).value)

There is a `CreateNamedStruct` expression (`named_struct`) to create a row of `InnerData.name` and `InnerData.value` that are referred by `ExternalMapToCatalyst_value52`.

Because `ExternalMapToCatalyst_value52` are local variable, when `CreateNamedStruct` splits expressions to individual functions, the local variable can't be accessed anymore.

## How was this patch tested?

Jenkins tests.

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

Closes #18418 from viirya/SPARK-19104.
2017-06-28 00:57:05 +08:00
Burak Yavuz 5282bae040 [SPARK-21153] Use project instead of expand in tumbling windows
## What changes were proposed in this pull request?

Time windowing in Spark currently performs an Expand + Filter, because there is no way to guarantee the amount of windows a timestamp will fall in, in the general case. However, for tumbling windows, a record is guaranteed to fall into a single bucket. In this case, doubling the number of records with Expand is wasteful, and can be improved by using a simple Projection instead.

Benchmarks show that we get an order of magnitude performance improvement after this patch.

## How was this patch tested?

Existing unit tests. Benchmarked using the following code:

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

spark.time {
  spark.range(numRecords)
    .select(from_unixtime((current_timestamp().cast("long") * 1000 + 'id / 1000) / 1000) as 'time)
    .select(window('time, "10 seconds"))
    .count()
}
```

Setup:
 - 1 c3.2xlarge worker (8 cores)

![image](https://user-images.githubusercontent.com/5243515/27348748-ed991b84-55a9-11e7-8f8b-6e7abc524417.png)

1 B rows ran in 287 seconds after this optimization. I didn't wait for it to finish without the optimization. Shows about 5x improvement for large number of records.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #18364 from brkyvz/opt-tumble.
2017-06-26 01:26:32 -07:00
gatorsmile 2e1586f60a [SPARK-21203][SQL] Fix wrong results of insertion of Array of Struct
### What changes were proposed in this pull request?
```SQL
CREATE TABLE `tab1`
(`custom_fields` ARRAY<STRUCT<`id`: BIGINT, `value`: STRING>>)
USING parquet

INSERT INTO `tab1`
SELECT ARRAY(named_struct('id', 1, 'value', 'a'), named_struct('id', 2, 'value', 'b'))

SELECT custom_fields.id, custom_fields.value FROM tab1
```

The above query always return the last struct of the array, because the rule `SimplifyCasts` incorrectly rewrites the query. The underlying cause is we always use the same `GenericInternalRow` object when doing the cast.

### How was this patch tested?

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18412 from gatorsmile/castStruct.
2017-06-24 22:35:59 +08:00
Xiao Li 03eb6117af [SPARK-21164][SQL] Remove isTableSample from Sample and isGenerated from Alias and AttributeReference
## What changes were proposed in this pull request?
`isTableSample` and `isGenerated ` were introduced for SQL Generation respectively by https://github.com/apache/spark/pull/11148 and https://github.com/apache/spark/pull/11050

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

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

Author: Xiao Li <gatorsmile@gmail.com>

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

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

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

Closes #17713 from dilipbiswal/subquery_checkanalysis.
2017-06-23 11:02:54 -07:00
Tathagata Das 2ebd0838d1 [SPARK-21192][SS] Preserve State Store provider class configuration across StreamingQuery restarts
## What changes were proposed in this pull request?

If the SQL conf for StateStore provider class is changed between restarts (i.e. query started with providerClass1 and attempted to restart using providerClass2), then the query will fail in a unpredictable way as files saved by one provider class cannot be used by the newer one.

Ideally, the provider class used to start the query should be used to restart the query, and the configuration in the session where it is being restarted should be ignored.

This PR saves the provider class config to OffsetSeqLog, in the same way # shuffle partitions is saved and recovered.

## How was this patch tested?
new unit tests

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

Closes #18402 from tdas/SPARK-21192.
2017-06-23 10:55:02 -07:00
wangzhenhua b803b66a81 [SPARK-21180][SQL] Remove conf from stats functions since now we have conf in LogicalPlan
## What changes were proposed in this pull request?

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

## How was this patch tested?

Covered by existing tests, plus some modified existing tests.

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

Closes #18391 from wzhfy/removeConf.
2017-06-23 10:33:53 -07:00
Takeshi Yamamuro f3dea60793 [SPARK-21144][SQL] Print a warning if the data schema and partition schema have the duplicate columns
## What changes were proposed in this pull request?
The current master outputs unexpected results when the data schema and partition schema have the duplicate columns:
```
withTempPath { dir =>
  val basePath = dir.getCanonicalPath
  spark.range(0, 3).toDF("foo").write.parquet(new Path(basePath, "foo=1").toString)
  spark.range(0, 3).toDF("foo").write.parquet(new Path(basePath, "foo=a").toString)
  spark.read.parquet(basePath).show()
}

+---+
|foo|
+---+
|  1|
|  1|
|  a|
|  a|
|  1|
|  a|
+---+
```
This patch added code to print a warning when the duplication found.

## How was this patch tested?
Manually checked.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #18375 from maropu/SPARK-21144-3.
2017-06-23 09:28:02 -07:00
Wang Gengliang b8a743b6a5 [SPARK-21174][SQL] Validate sampling fraction in logical operator level
## What changes were proposed in this pull request?

Currently the validation of sampling fraction in dataset is incomplete.
As an improvement, validate sampling fraction in logical operator level:
1) if with replacement: fraction should be nonnegative
2) else: fraction should be on interval [0, 1]
Also add test cases for the validation.

## How was this patch tested?
integration tests

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

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #18387 from gengliangwang/sample_ratio_validate.
2017-06-23 09:27:35 +08:00
Bryan Cutler e44697606f [SPARK-13534][PYSPARK] Using Apache Arrow to increase performance of DataFrame.toPandas
## What changes were proposed in this pull request?
Integrate Apache Arrow with Spark to increase performance of `DataFrame.toPandas`.  This has been done by using Arrow to convert data partitions on the executor JVM to Arrow payload byte arrays where they are then served to the Python process.  The Python DataFrame can then collect the Arrow payloads where they are combined and converted to a Pandas DataFrame.  All non-complex data types are currently supported, otherwise an `UnsupportedOperation` exception is thrown.

Additions to Spark include a Scala package private method `Dataset.toArrowPayloadBytes` that will convert data partitions in the executor JVM to `ArrowPayload`s as byte arrays so they can be easily served.  A package private class/object `ArrowConverters` that provide data type mappings and conversion routines.  In Python, a public method `DataFrame.collectAsArrow` is added to collect Arrow payloads and an optional flag in `toPandas(useArrow=False)` to enable using Arrow (uses the old conversion by default).

## How was this patch tested?
Added a new test suite `ArrowConvertersSuite` that will run tests on conversion of Datasets to Arrow payloads for supported types.  The suite will generate a Dataset and matching Arrow JSON data, then the dataset is converted to an Arrow payload and finally validated against the JSON data.  This will ensure that the schema and data has been converted correctly.

Added PySpark tests to verify the `toPandas` method is producing equal DataFrames with and without pyarrow.  A roundtrip test to ensure the pandas DataFrame produced by pyspark is equal to a one made directly with pandas.

Author: Bryan Cutler <cutlerb@gmail.com>
Author: Li Jin <ice.xelloss@gmail.com>
Author: Li Jin <li.jin@twosigma.com>
Author: Wes McKinney <wes.mckinney@twosigma.com>

Closes #15821 from BryanCutler/wip-toPandas_with_arrow-SPARK-13534.
2017-06-23 09:01:13 +08:00
Xingbo Jiang cad88f17e8 [SPARK-17851][SQL][TESTS] Make sure all test sqls in catalyst pass checkAnalysis
## What changes were proposed in this pull request?

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

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

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

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

Closes #15417 from jiangxb1987/cptest.
2017-06-21 09:40:06 -07:00
Marcos P e92befcb4b [MINOR][DOC] modified issue link and updated status
## What changes were proposed in this pull request?

This PR aims to clarify some outdated comments that i found at **spark-catalyst** and **spark-sql** pom files. Maven bug still happening and in order to track it I have updated the issue link and also the status of the issue.

Author: Marcos P <mpenate@stratio.com>

Closes #18374 from mpenate/fix/mng-3559-comment.
2017-06-21 15:34:10 +01:00
Reynold Xin b6b108826a [SPARK-21103][SQL] QueryPlanConstraints should be part of LogicalPlan
## What changes were proposed in this pull request?
QueryPlanConstraints should be part of LogicalPlan, rather than QueryPlan, since the constraint framework is only used for query plan rewriting and not for physical planning.

## How was this patch tested?
Should be covered by existing tests, since it is a simple refactoring.

Author: Reynold Xin <rxin@databricks.com>

Closes #18310 from rxin/SPARK-21103.
2017-06-20 11:34:22 -07:00
Xianyang Liu 0a4b7e4f81 [MINOR] Fix some typo of the document
## What changes were proposed in this pull request?

Fix some typo of the document.

## How was this patch tested?

Existing tests.

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

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

Closes #18350 from ConeyLiu/fixtypo.
2017-06-19 20:35:58 +01:00
Dongjoon Hyun ecc5631351 [MINOR][BUILD] Fix Java linter errors
## What changes were proposed in this pull request?

This PR cleans up a few Java linter errors for Apache Spark 2.2 release.

## How was this patch tested?

```bash
$ dev/lint-java
Using `mvn` from path: /usr/local/bin/mvn
Checkstyle checks passed.
```

We can check the result at Travis CI, [here](https://travis-ci.org/dongjoon-hyun/spark/builds/244297894).

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #18345 from dongjoon-hyun/fix_lint_java_2.
2017-06-19 20:17:54 +01:00
Xiao Li 9413b84b5a [SPARK-21132][SQL] DISTINCT modifier of function arguments should not be silently ignored
### What changes were proposed in this pull request?
We should not silently ignore `DISTINCT` when they are not supported in the function arguments. This PR is to block these cases and issue the error messages.

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

Author: Xiao Li <gatorsmile@gmail.com>

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

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

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18173 from wangyum/SPARK-20948.
2017-06-18 20:14:05 -07:00
Yuming Wang ce49428ef7 [SPARK-20749][SQL][FOLLOWUP] Support character_length
## What changes were proposed in this pull request?

The function `char_length` is shorthand for `character_length` function. Both Hive and Postgresql support `character_length`,  This PR add support for `character_length`.

Ref:
https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF#LanguageManualUDF-StringFunctions
https://www.postgresql.org/docs/current/static/functions-string.html

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

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

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

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

## How was this patch tested?
 unit tests

Author: Yuming Wang <wgyumg@gmail.com>

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

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

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

## How was this patch tested?

Added new test suites for these three functions

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

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

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

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

## How was this patch tested?

Existing tests.

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

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

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

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

## How was this patch tested?

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

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

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

Closes #18075 from bdrillard/class_splitting_only.
2017-06-15 13:45:08 +08:00
Xiao Li 2051428173 [SPARK-20980][SQL] Rename wholeFile to multiLine for both CSV and JSON
### What changes were proposed in this pull request?
The current option name `wholeFile` is misleading for CSV users. Currently, it is not representing a record per file. Actually, one file could have multiple records. Thus, we should rename it. Now, the proposal is `multiLine`.

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

Author: Xiao Li <gatorsmile@gmail.com>

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

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

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

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

Author: Reynold Xin <rxin@databricks.com>

Closes #18299 from rxin/SPARK-21092.
2017-06-14 22:11:41 -07:00
Reynold Xin e254e868f1 [SPARK-21091][SQL] Move constraint code into QueryPlanConstraints
## What changes were proposed in this pull request?
This patch moves constraint related code into a separate trait QueryPlanConstraints, so we don't litter QueryPlan with a lot of constraint private functions.

## How was this patch tested?
This is a simple move refactoring and should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #18298 from rxin/SPARK-21091.
2017-06-14 14:28:21 -07:00
Xiao Li 77a2fc5b52 Revert "[SPARK-20941][SQL] Fix SubqueryExec Reuse"
This reverts commit f7cf2096fd.
2017-06-14 11:48:32 -07:00
Xiao Li df766a4714 [SPARK-21089][SQL] Fix DESC EXTENDED/FORMATTED to Show Table Properties
### What changes were proposed in this pull request?

Since both table properties and storage properties share the same key values, table properties are not shown in the output of DESC EXTENDED/FORMATTED when the storage properties are not empty.

This PR is to fix the above issue by renaming them to different keys.

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18294 from gatorsmile/tableProperties.
2017-06-14 11:13:16 -07:00
Yuming Wang 4d01aa4648 [SPARK-20754][SQL][FOLLOWUP] Add Function Alias For MOD/POSITION.
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

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

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

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

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

## How was this patch tested?

Pass the Jenkins with a new test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

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

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

Author: Reynold Xin <rxin@databricks.com>

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

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

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

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

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

 add test case to MathExpressionsSuite as #17906

## How was this patch tested?

unit test cases

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

Closes #18082 from 10110346/wip-lx-0524.
2017-06-11 22:29:09 -07:00
Michal Senkyr f48273c13c [SPARK-18891][SQL] Support for specific Java List subtypes
## What changes were proposed in this pull request?

Add support for specific Java `List` subtypes in deserialization as well as a generic implicit encoder.

All `List` subtypes are supported by using either the size-specifying constructor (one `int` parameter) or the default constructor.

Interfaces/abstract classes use the following implementations:

* `java.util.List`, `java.util.AbstractList` or `java.util.AbstractSequentialList` => `java.util.ArrayList`

## How was this patch tested?

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

Additionally in Spark shell:

```
scala> val jlist = new java.util.LinkedList[Int]; jlist.add(1)
jlist: java.util.LinkedList[Int] = [1]
res0: Boolean = true

scala> Seq(jlist).toDS().map(_.element()).collect()
res1: Array[Int] = Array(1)
```

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

Closes #18009 from michalsenkyr/dataset-java-lists.
2017-06-12 08:53:23 +08:00
Michal Senkyr 0538f3b0ae [SPARK-18891][SQL] Support for Scala Map collection types
## What changes were proposed in this pull request?

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

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

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

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

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

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

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

Codegen for `java.util.Map`:

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

## How was this patch tested?

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

Additionally in Spark shell:

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

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

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

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

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

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

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

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

## How was this patch tested?

Added new tests.

Author: Zhenhua Wang <wzh_zju@163.com>

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

## How was this patch tested?
unit tests

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

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

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

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

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

Author: Xiao Li <gatorsmile@gmail.com>

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

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

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

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

Closes #18142 from gatorsmile/fuctionRegistry.
2017-06-09 10:16:30 -07:00
Xiao Li 1a527bde49 [SPARK-20976][SQL] Unify Error Messages for FAILFAST mode
### What changes were proposed in this pull request?
Before 2.2, we indicate the job was terminated because of `FAILFAST` mode.
```
Malformed line in FAILFAST mode: {"a":{, b:3}
```
If possible, we should keep it. This PR is to unify the error messages.

### How was this patch tested?
Modified the existing messages.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18196 from gatorsmile/messFailFast.
2017-06-08 12:10:31 -07:00
Bogdan Raducanu cb83ca1433 [SPARK-20854][TESTS] Removing duplicate test case
## What changes were proposed in this pull request?

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

## How was this patch tested?
Existing tests.

Author: Bogdan Raducanu <bogdan@databricks.com>

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

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

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

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18189 from cloud-fan/stats.
2017-06-06 22:50:06 -07:00
Reza Safi b61a401da8 [SPARK-20926][SQL] Removing exposures to guava library caused by directly accessing SessionCatalog's tableRelationCache
There could be test failures because DataStorageStrategy, HiveMetastoreCatalog and also HiveSchemaInferenceSuite were exposed to guava library by directly accessing SessionCatalog's tableRelationCacheg. These failures occur when guava shading is in place.

## What changes were proposed in this pull request?
This change removes those guava exposures by introducing new methods in SessionCatalog and also changing DataStorageStrategy, HiveMetastoreCatalog and HiveSchemaInferenceSuite so that they use those proxy methods.

## How was this patch tested?

Unit tests passed after applying these changes.

Author: Reza Safi <rezasafi@cloudera.com>

Closes #18148 from rezasafi/branch-2.2.

(cherry picked from commit 1388fdd707)
2017-06-06 09:54:13 -07:00
Feng Liu 88a23d3de0 [SPARK-20991][SQL] BROADCAST_TIMEOUT conf should be a TimeoutConf
## What changes were proposed in this pull request?

The construction of BROADCAST_TIMEOUT conf should take the TimeUnit argument as a TimeoutConf.

Author: Feng Liu <fengliu@databricks.com>

Closes #18208 from liufengdb/fix_timeout.
2017-06-05 17:48:28 -07:00
Wieland Hoffmann c70c38eb93 [DOCS] Fix a typo in Encoder.clsTag
## What changes were proposed in this pull request?

Fixes a typo: `and` -> `an`

## How was this patch tested?

Not at all.

Author: Wieland Hoffmann <mineo@users.noreply.github.com>

Closes #17759 from mineo/patch-1.
2017-06-03 10:12:37 +01:00
Xiao Li 2a780ac7fe [MINOR][SQL] Update the description of spark.sql.files.ignoreCorruptFiles and spark.sql.columnNameOfCorruptRecord
### What changes were proposed in this pull request?
1. The description of `spark.sql.files.ignoreCorruptFiles` is not accurate. When the file does not exist, we will issue the error message.
```
org.apache.spark.sql.AnalysisException: Path does not exist: file:/nonexist/path;
```

2. `spark.sql.columnNameOfCorruptRecord` also affects the CSV format. The current description only mentions JSON format.

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18184 from gatorsmile/updateMessage.
2017-06-02 12:58:29 -07:00
Bogdan Raducanu 2134196a9c [SPARK-20854][SQL] Extend hint syntax to support expressions
## What changes were proposed in this pull request?

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

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

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

Author: Bogdan Raducanu <bogdan@databricks.com>

Closes #18086 from bogdanrdc/SPARK-20854.
2017-06-01 15:50:40 -07:00
Xiao Li f7cf2096fd [SPARK-20941][SQL] Fix SubqueryExec Reuse
### What changes were proposed in this pull request?
Before this PR, Subquery reuse does not work. Below are three issues:
- Subquery reuse does not work.
- It is sharing the same `SQLConf` (`spark.sql.exchange.reuse`) with the one for Exchange Reuse.
- No test case covers the rule Subquery reuse.

This PR is to fix the above three issues.
- Ignored the physical operator `SubqueryExec` when comparing two plans.
- Added a dedicated conf `spark.sql.subqueries.reuse` for controlling Subquery Reuse
- Added a test case for verifying the behavior

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18169 from gatorsmile/subqueryReuse.
2017-06-01 09:52:18 -07:00
Yuming Wang 6d05c1c1da [SPARK-20910][SQL] Add build-in SQL function - UUID
## What changes were proposed in this pull request?

Add build-int SQL function - UUID.

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18136 from wangyum/SPARK-20910.
2017-06-01 16:15:24 +09:00
Yuming Wang c8045f8b48 [MINOR][SQL] Fix a few function description error.
## What changes were proposed in this pull request?

Fix a few function description error.

## How was this patch tested?

manual tests

![descissues](https://cloud.githubusercontent.com/assets/5399861/26619392/d547736c-4610-11e7-85d7-aeeb09c02cc8.gif)

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18157 from wangyum/DescIssues.
2017-05-31 23:17:15 -07:00
Jacek Laskowski beed5e20af [DOCS][MINOR] Scaladoc fixes (aka typo hunting)
## What changes were proposed in this pull request?

Minor changes to scaladoc

## How was this patch tested?

Local build

Author: Jacek Laskowski <jacek@japila.pl>

Closes #18074 from jaceklaskowski/scaladoc-fixes.
2017-05-31 11:24:37 +01:00
Wenchen Fan 1f5dddffa3 Revert "[SPARK-20392][SQL] Set barrier to prevent re-entering a tree"
This reverts commit 8ce0d8ffb6.
2017-05-30 21:14:55 -07:00
Wenchen Fan 10e526e7e6 [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL UI tab
## What changes were proposed in this pull request?

Currently the `DataFrameWriter` operations have several problems:

1. non-file-format data source writing action doesn't show up in the SQL tab in Spark UI
2. file-format data source writing action shows a scan node in the SQL tab, without saying anything about writing. (streaming also have this issue, but not fixed in this PR)
3. Spark SQL CLI actions don't show up in the SQL tab.

This PR fixes all of them, by refactoring the `ExecuteCommandExec` to make it have children.

 close https://github.com/apache/spark/pull/17540

## How was this patch tested?

existing tests.

Also test the UI manually. For a simple command: `Seq(1 -> "a").toDF("i", "j").write.parquet("/tmp/qwe")`

before this PR:
<img width="266" alt="qq20170523-035840 2x" src="https://cloud.githubusercontent.com/assets/3182036/26326050/24e18ba2-3f6c-11e7-8817-6dd275bf6ac5.png">
after this PR:
<img width="287" alt="qq20170523-035708 2x" src="https://cloud.githubusercontent.com/assets/3182036/26326054/2ad7f460-3f6c-11e7-8053-d68325beb28f.png">

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18064 from cloud-fan/execution.
2017-05-30 20:12:32 -07:00
Tathagata Das fa757ee1d4 [SPARK-20883][SPARK-20376][SS] Refactored StateStore APIs and added conf to choose implementation
## What changes were proposed in this pull request?

A bunch of changes to the StateStore APIs and implementation.
Current state store API has a bunch of problems that causes too many transient objects causing memory pressure.

- `StateStore.get(): Option` forces creation of Some/None objects for every get. Changed this to return the row or null.
- `StateStore.iterator(): (UnsafeRow, UnsafeRow)` forces creation of new tuple for each record returned. Changed this to return a UnsafeRowTuple which can be reused across records.
- `StateStore.updates()` requires the implementation to keep track of updates, while this is used minimally (only by Append mode in streaming aggregations). Removed updates() and updated StateStoreSaveExec accordingly.
- `StateStore.filter(condition)` and `StateStore.remove(condition)` has been merge into a single API `getRange(start, end)` which allows a state store to do optimized range queries (i.e. avoid full scans). Stateful operators have been updated accordingly.
- Removed a lot of unnecessary row copies Each operator copied rows before calling StateStore.put() even if the implementation does not require it to be copied. It is left up to the implementation on whether to copy the row or not.

Additionally,
- Added a name to the StateStoreId so that each operator+partition can use multiple state stores (different names)
- Added a configuration that allows the user to specify which implementation to use.
- Added new metrics to understand the time taken to update keys, remove keys and commit all changes to the state store. These metrics will be visible on the plan diagram in the SQL tab of the UI.
- Refactored unit tests such that they can be reused to test any implementation of StateStore.

## How was this patch tested?
Old and new unit tests

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

Closes #18107 from tdas/SPARK-20376.
2017-05-30 15:33:06 -07:00
Xiao Li 4bb6a53ebd [SPARK-20924][SQL] Unable to call the function registered in the not-current database
### What changes were proposed in this pull request?
We are unable to call the function registered in the not-current database.
```Scala
sql("CREATE DATABASE dAtABaSe1")
sql(s"CREATE FUNCTION dAtABaSe1.test_avg AS '${classOf[GenericUDAFAverage].getName}'")
sql("SELECT dAtABaSe1.test_avg(1)")
```
The above code returns an error:
```
Undefined function: 'dAtABaSe1.test_avg'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 7
```

This PR is to fix the above issue.
### How was this patch tested?
Added test cases.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18146 from gatorsmile/qualifiedFunction.
2017-05-30 14:06:19 -07:00
Liang-Chi Hsieh 35b644bd03 [SPARK-20916][SQL] Improve error message for unaliased subqueries in FROM clause
## What changes were proposed in this pull request?

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

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

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

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

## How was this patch tested?

Modified existing tests to reflect this change.

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

Closes #18141 from viirya/SPARK-20916.
2017-05-30 06:28:43 -07:00
Yuming Wang 80fb24b85d [MINOR] Fix some indent issues.
## What changes were proposed in this pull request?

Fix some indent issues.

## How was this patch tested?

existing tests.

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18133 from wangyum/IndentIssues.
2017-05-30 12:15:54 +01:00
Yuming Wang d797ed0ef1 [SPARK-20909][SQL] Add build-int SQL function - DAYOFWEEK
## What changes were proposed in this pull request?

Add build-int SQL function - DAYOFWEEK

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18134 from wangyum/SPARK-20909.
2017-05-30 15:40:50 +09:00
Yuming Wang 1c7db00c74 [SPARK-8184][SQL] Add additional function description for weekofyear
## What changes were proposed in this pull request?

Add additional function description for weekofyear.

## How was this patch tested?

 manual tests

![weekofyear](https://cloud.githubusercontent.com/assets/5399861/26525752/08a1c278-4394-11e7-8988-7cbf82c3a999.gif)

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18132 from wangyum/SPARK-8184.
2017-05-29 16:10:22 -07:00
Kazuaki Ishizaki ef9fd920c3 [SPARK-20750][SQL] Built-in SQL Function Support - REPLACE
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

added new test suites

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

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

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

## How was this patch tested?

- Added unit tests

Author: Tejas Patil <tejasp@fb.com>

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

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

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

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

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

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

Author: Xiao Li <gatorsmile@gmail.com>

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

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

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

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

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

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

## How was this patch tested?
unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18019 from wangyum/SPARK-20748.
2017-05-26 20:59:14 -07:00
Michael Armbrust d935e0a9d9 [SPARK-20844] Remove experimental from Structured Streaming APIs
Now that Structured Streaming has been out for several Spark release and has large production use cases, the `Experimental` label is no longer appropriate.  I've left `InterfaceStability.Evolving` however, as I think we may make a few changes to the pluggable Source & Sink API in Spark 2.3.

Author: Michael Armbrust <michael@databricks.com>

Closes #18065 from marmbrus/streamingGA.
2017-05-26 13:33:23 -07:00
Liang-Chi Hsieh 8ce0d8ffb6 [SPARK-20392][SQL] Set barrier to prevent re-entering a tree
## What changes were proposed in this pull request?

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

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

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

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

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

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

### Performance improvement

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

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

## How was this patch tested?

Existing tests.

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

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

Closes #17770 from viirya/SPARK-20392.
2017-05-26 13:45:55 +08:00
liuxian 197f9018a4 [SPARK-20403][SQL] Modify the instructions of some functions
## What changes were proposed in this pull request?
1.    add  instructions of  'cast'  function When using 'show functions'  and 'desc function cast'
       command in spark-sql
2.    Modify the  instructions of functions,such as
     boolean,tinyint,smallint,int,bigint,float,double,decimal,date,timestamp,binary,string

## How was this patch tested?
Before modification:
spark-sql>desc function boolean;
Function: boolean
Class: org.apache.spark.sql.catalyst.expressions.Cast
Usage: boolean(expr AS type) - Casts the value `expr` to the target data type `type`.

After modification:
spark-sql> desc function boolean;
Function: boolean
Class: org.apache.spark.sql.catalyst.expressions.Cast
Usage: boolean(expr) - Casts the value `expr` to the target data type `boolean`.

spark-sql> desc function cast
Function: cast
Class: org.apache.spark.sql.catalyst.expressions.Cast
Usage: cast(expr AS type) - Casts the value `expr` to the target data type `type`.

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

Closes #17698 from 10110346/wip_lx_0418.
2017-05-24 17:32:02 -07:00
Reynold Xin a64746677b [SPARK-20867][SQL] Move hints from Statistics into HintInfo class
## What changes were proposed in this pull request?
This is a follow-up to SPARK-20857 to move the broadcast hint from Statistics into a new HintInfo class, so we can be more flexible in adding new hints in the future.

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

Author: Reynold Xin <rxin@databricks.com>

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

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

Author: Reynold Xin <rxin@databricks.com>

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

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

This follow-up is going to fix the tests.

## How was this patch tested?

Jenkins tests.

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

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

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

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

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

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

Added a few test cases.

Author: Xiao Li <gatorsmile@gmail.com>

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

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

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

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

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

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

Author: gatorsmile <gatorsmile@gmail.com>

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

spark-sql>SELECT ceil(1234567890123456);
1234567890123456

spark-sql>SELECT ceil(12345678901234567);
12345678901234568

spark-sql>SELECT ceil(123456789012345678);
123456789012345680

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

but mysql handle the value is ok.

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

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

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

## How was this patch tested?

Supplement the unit test.

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

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

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

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

## How was this patch tested?
unit tests

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

Closes #17997 from 10110346/wip_lx_0516.
2017-05-19 10:25:21 -07:00
Yuming Wang bff021dfaf [SPARK-20751][SQL] Add built-in SQL Function - COT
## What changes were proposed in this pull request?

Add built-in SQL Function - COT.

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #17999 from wangyum/SPARK-20751.
2017-05-19 09:40:22 -07:00
Ala Luszczak ce8edb8bf4 [SPARK-20798] GenerateUnsafeProjection should check if a value is null before calling the getter
## What changes were proposed in this pull request?

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

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

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

## How was this patch tested?

Adds GenerateUnsafeProjectionSuite.

Author: Ala Luszczak <ala@databricks.com>

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

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

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

## How was this patch tested?

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

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

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

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

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

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

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

## How was this patch tested?

Jenkins tests.

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

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

Closes #17935 from viirya/SPARK-20690.
2017-05-17 12:57:35 +08:00
Herman van Hovell 69bb7715f9 [SQL][TRIVIAL] Lower parser log level to debug
## What changes were proposed in this pull request?
Currently the parser logs the query it is parsing at `info` level. This is too high, this PR lowers the log level to `debug`.

## How was this patch tested?
Existing tests.

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

Closes #18006 from hvanhovell/lower_parser_log_level.
2017-05-16 15:58:50 -07:00
Kazuaki Ishizaki 6f62e9d9b9 [SPARK-19372][SQL] Fix throwing a Java exception at df.fliter() due to 64KB bytecode size limit
## What changes were proposed in this pull request?

When an expression for `df.filter()` has many nodes (e.g. 400), the size of Java bytecode for the generated Java code is more than 64KB. It produces an Java exception. As a result, the execution fails.
This PR continues to execute by calling `Expression.eval()` disabling code generation if an exception has been caught.

## How was this patch tested?

Add a test suite into `DataFrameSuite`

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

Closes #17087 from kiszk/SPARK-19372.
2017-05-16 14:47:21 -07:00
Takuya UESHIN c8c878a416 [SPARK-20588][SQL] Cache TimeZone instances.
## What changes were proposed in this pull request?

Because the method `TimeZone.getTimeZone(String ID)` is synchronized on the TimeZone class, concurrent call of this method will become a bottleneck.
This especially happens when casting from string value containing timezone info to timestamp value, which uses `DateTimeUtils.stringToTimestamp()` and gets TimeZone instance on the site.

This pr makes a cache of the generated TimeZone instances to avoid the synchronization.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #17933 from ueshin/issues/SPARK-20588.
2017-05-15 16:52:22 -07:00
Tejas Patil d2416925c4 [SPARK-17729][SQL] Enable creating hive bucketed tables
## What changes were proposed in this pull request?

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

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

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

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

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

Author: Tejas Patil <tejasp@fb.com>

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

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17970 from maropu/SPARK-20730.
2017-05-15 16:24:55 +08:00
Wenchen Fan 1283c3d11a [SPARK-20725][SQL] partial aggregate should behave correctly for sameResult
## What changes were proposed in this pull request?

For aggregate function with `PartialMerge` or `Final` mode, the input is aggregate buffers instead of the actual children expressions. So the actual children expressions won't affect the result, we should normalize the expr id for them.

## How was this patch tested?

a new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17964 from cloud-fan/tmp.
2017-05-13 12:09:06 -07:00
hyukjinkwon 3f98375d8a [SPARK-18772][SQL] Avoid unnecessary conversion try for special floats in JSON
## What changes were proposed in this pull request?

This PR is based on  https://github.com/apache/spark/pull/16199 and extracts the valid change from https://github.com/apache/spark/pull/9759 to resolve SPARK-18772

This avoids additional conversion try with `toFloat` and `toDouble`.

For avoiding additional conversions, please refer the codes below:

**Before**

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

scala> spark.read.schema(StructType(Seq(StructField("a", DoubleType)))).option("mode", "FAILFAST").json(Seq("""{"a": "nan"}""").toDS).show()
17/05/12 11:30:41 ERROR Executor: Exception in task 0.0 in stage 2.0 (TID 2)
java.lang.NumberFormatException: For input string: "nan"
...
```

**After**

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

scala> spark.read.schema(StructType(Seq(StructField("a", DoubleType)))).option("mode", "FAILFAST").json(Seq("""{"a": "nan"}""").toDS).show()
17/05/12 11:44:30 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0)
java.lang.RuntimeException: Cannot parse nan as DoubleType.
...
```

## How was this patch tested?

Unit tests added in `JsonSuite`.

Closes #16199

Author: hyukjinkwon <gurwls223@gmail.com>
Author: Nathan Howell <nhowell@godaddy.com>

Closes #17956 from HyukjinKwon/SPARK-18772.
2017-05-13 20:56:04 +08:00
Xiao Li b84ff7eb62 [SPARK-20719][SQL] Support LIMIT ALL
### What changes were proposed in this pull request?
`LIMIT ALL` is the same as omitting the `LIMIT` clause. It is supported by both PrestgreSQL and Presto. This PR is to support it by adding it in the parser.

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17960 from gatorsmile/LimitAll.
2017-05-12 15:26:10 -07:00
Takeshi Yamamuro b526f70c16 [SPARK-19951][SQL] Add string concatenate operator || to Spark SQL
## What changes were proposed in this pull request?
This pr added code to support `||` for string concatenation. This string operation is supported in PostgreSQL and MySQL.

## How was this patch tested?
Added tests in `SparkSqlParserSuite`

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17711 from maropu/SPARK-19951.
2017-05-12 09:55:51 -07:00
Takeshi Yamamuro 92ea7fd7b6 [SPARK-20710][SQL] Support aliases in CUBE/ROLLUP/GROUPING SETS
## What changes were proposed in this pull request?
This pr added  `Analyzer` code for supporting aliases in CUBE/ROLLUP/GROUPING SETS (This is follow-up of #17191).

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17948 from maropu/SPARK-20710.
2017-05-12 20:48:30 +08:00
wangzhenhua 54b4f2ad43 [SPARK-20718][SQL][FOLLOWUP] Fix canonicalization for HiveTableScanExec
## What changes were proposed in this pull request?

Fix canonicalization for different filter orders in `HiveTableScanExec`.

## How was this patch tested?

Added a new test case.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17962 from wzhfy/canonicalizeHiveTableScanExec.
2017-05-12 20:43:22 +08:00
Ryan Blue b236933907 [SPARK-17424] Fix unsound substitution bug in ScalaReflection.
## What changes were proposed in this pull request?

This method gets a type's primary constructor and fills in type parameters with concrete types. For example, `MapPartitions[T, U] -> MapPartitions[Int, String]`. This Substitution fails when the actual type args are empty because they are still unknown. Instead, when there are no resolved types to subsitute, this returns the original args with unresolved type parameters.
## How was this patch tested?

This doesn't affect substitutions where the type args are determined. This fixes our case where the actual type args are empty and our job runs successfully.

Author: Ryan Blue <blue@apache.org>

Closes #15062 from rdblue/SPARK-17424-fix-unsound-reflect-substitution.
2017-05-12 20:38:36 +08:00
hyukjinkwon 720708ccdd [SPARK-20639][SQL] Add single argument support for to_timestamp in SQL with documentation improvement
## What changes were proposed in this pull request?

This PR proposes three things as below:

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

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

  For example, the one below works

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

  prints

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

  whereas this does not work in SQL.

  **Before**

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

  **After**

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

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

  **Before**

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

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

  **After**

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

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

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

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

## How was this patch tested?

Added tests in `datetime.sql`.

Author: hyukjinkwon <gurwls223@gmail.com>

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

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

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

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

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

The following codes can reproduce it:

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

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

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

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

## How was this patch tested?

Jenkins tests.

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

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

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

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17928 from maropu/SPARK-20311-3.
2017-05-11 18:09:31 +08:00
Wenchen Fan b4c99f4369 [SPARK-20569][SQL] RuntimeReplaceable functions should not take extra parameters
## What changes were proposed in this pull request?

`RuntimeReplaceable` always has a constructor with the expression to replace with, and this constructor should not be the function builder.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17876 from cloud-fan/minor.
2017-05-11 00:41:15 -07:00
wangzhenhua 76e4a5566b [SPARK-20678][SQL] Ndv for columns not in filter condition should also be updated
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

Added new tests.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17918 from wzhfy/scaleDownNdvAfterFilter.
2017-05-10 19:42:49 +08:00
Wenchen Fan 789bdbe3d0 [SPARK-20688][SQL] correctly check analysis for scalar sub-queries
## What changes were proposed in this pull request?

In `CheckAnalysis`, we should call `checkAnalysis` for `ScalarSubquery` at the beginning, as later we will call `plan.output` which is invalid if `plan` is not resolved.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17930 from cloud-fan/tmp.
2017-05-10 19:30:00 +08:00
Takuya UESHIN 0ef16bd4b0 [SPARK-20668][SQL] Modify ScalaUDF to handle nullability.
## What changes were proposed in this pull request?

When registering Scala UDF, we can know if the udf will return nullable value or not. `ScalaUDF` and related classes should handle the nullability.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #17911 from ueshin/issues/SPARK-20668.
2017-05-09 23:48:25 -07:00
Josh Rosen a90c5cd822 [SPARK-20686][SQL] PropagateEmptyRelation incorrectly handles aggregate without grouping
## What changes were proposed in this pull request?

The query

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

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

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

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

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

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

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

## How was this patch tested?

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

Author: Josh Rosen <joshrosen@databricks.com>

Closes #17929 from JoshRosen/fix-PropagateEmptyRelation.
2017-05-10 14:36:36 +08:00
uncleGen c0189abc7c [SPARK-20373][SQL][SS] Batch queries with 'Dataset/DataFrame.withWatermark()` does not execute
## What changes were proposed in this pull request?

Any Dataset/DataFrame batch query with the operation `withWatermark` does not execute because the batch planner does not have any rule to explicitly handle the EventTimeWatermark logical plan.
The right solution is to simply remove the plan node, as the watermark should not affect any batch query in any way.

Changes:
- In this PR, we add a new rule `EliminateEventTimeWatermark` to check if we need to ignore the event time watermark. We will ignore watermark in any batch query.

Depends upon:
- [SPARK-20672](https://issues.apache.org/jira/browse/SPARK-20672). We can not add this rule into analyzer directly, because streaming query will be copied to `triggerLogicalPlan ` in every trigger, and the rule will be applied to `triggerLogicalPlan` mistakenly.

Others:
- A typo fix in example.

## How was this patch tested?

add new unit test.

Author: uncleGen <hustyugm@gmail.com>

Closes #17896 from uncleGen/SPARK-20373.
2017-05-09 15:08:09 -07:00
Yin Huai f79aa285cf Revert "[SPARK-20311][SQL] Support aliases for table value functions"
This reverts commit 714811d0b5.
2017-05-09 14:47:45 -07:00
Reynold Xin ac1ab6b9db Revert "[SPARK-12297][SQL] Hive compatibility for Parquet Timestamps"
This reverts commit 22691556e5.

See JIRA ticket for more information.
2017-05-09 11:35:59 -07:00
Takeshi Yamamuro 714811d0b5 [SPARK-20311][SQL] Support aliases for table value functions
## What changes were proposed in this pull request?
This pr added parsing rules to support aliases in table value functions.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17666 from maropu/SPARK-20311.
2017-05-09 20:22:51 +08:00
Xiao Li 0d00c768a8 [SPARK-20667][SQL][TESTS] Cleanup the cataloged metadata after completing the package of sql/core and sql/hive
## What changes were proposed in this pull request?

So far, we do not drop all the cataloged objects after each package. Sometimes, we might hit strange test case errors because the previous test suite did not drop the cataloged/temporary objects (tables/functions/database). At least, we can first clean up the environment when completing the package of `sql/core` and `sql/hive`.

## How was this patch tested?
N/A

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17908 from gatorsmile/reset.
2017-05-09 20:10:50 +08:00
sujith71955 42cc6d13ed [SPARK-20380][SQL] Unable to set/unset table comment property using ALTER TABLE SET/UNSET TBLPROPERTIES ddl
### What changes were proposed in this pull request?
Table comment was not getting  set/unset using **ALTER TABLE  SET/UNSET TBLPROPERTIES** query
eg: ALTER TABLE table_with_comment SET TBLPROPERTIES("comment"= "modified comment)
 when user alter the table properties  and adds/updates table comment,table comment which is a field  of **CatalogTable**  instance is not getting updated and  old table comment if exists was shown to user, inorder  to handle this issue, update the comment field value in **CatalogTable** with the newly added/modified comment along with other table level properties when user executes **ALTER TABLE  SET TBLPROPERTIES** query.

This pr has also taken care of unsetting the table comment when user executes query  **ALTER TABLE  UNSET TBLPROPERTIES** inorder to unset or remove table comment.
eg: ALTER TABLE table_comment UNSET TBLPROPERTIES IF EXISTS ('comment')

### How was this patch tested?
Added test cases  as part of **SQLQueryTestSuite** for verifying  table comment using desc formatted table query after adding/modifying table comment as part of **AlterTableSetPropertiesCommand** and unsetting the table comment using **AlterTableUnsetPropertiesCommand**.

Author: sujith71955 <sujithchacko.2010@gmail.com>

Closes #17649 from sujith71955/alter_table_comment.
2017-05-07 23:15:00 -07:00
Imran Rashid 22691556e5 [SPARK-12297][SQL] Hive compatibility for Parquet Timestamps
## What changes were proposed in this pull request?

This change allows timestamps in parquet-based hive table to behave as a "floating time", without a timezone, as timestamps are for other file formats.  If the storage timezone is the same as the session timezone, this conversion is a no-op.  When data is read from a hive table, the table property is *always* respected.  This allows spark to not change behavior when reading old data, but read newly written data correctly (whatever the source of the data is).

Spark inherited the original behavior from Hive, but Hive is also updating behavior to use the same  scheme in HIVE-12767 / HIVE-16231.

The default for Spark remains unchanged; created tables do not include the new table property.

This will only apply to hive tables; nothing is added to parquet metadata to indicate the timezone, so data that is read or written directly from parquet files will never have any conversions applied.

## How was this patch tested?

Added a unit test which creates tables, reads and writes data, under a variety of permutations (different storage timezones, different session timezones, vectorized reading on and off).

Author: Imran Rashid <irashid@cloudera.com>

Closes #16781 from squito/SPARK-12297.
2017-05-08 12:16:00 +09:00
Jacek Laskowski 500436b436 [MINOR][SQL][DOCS] Improve unix_timestamp's scaladoc (and typo hunting)
## What changes were proposed in this pull request?

* Docs are consistent (across different `unix_timestamp` variants and their internal expressions)
* typo hunting

## How was this patch tested?

local build

Author: Jacek Laskowski <jacek@japila.pl>

Closes #17801 from jaceklaskowski/unix_timestamp.
2017-05-07 13:56:13 -07:00
Juliusz Sompolski 5d75b14bf0 [SPARK-20616] RuleExecutor logDebug of batch results should show diff to start of batch
## What changes were proposed in this pull request?

Due to a likely typo, the logDebug msg printing the diff of query plans shows a diff to the initial plan, not diff to the start of batch.

## How was this patch tested?

Now the debug message prints the diff between start and end of batch.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #17875 from juliuszsompolski/SPARK-20616.
2017-05-05 15:31:06 -07:00
Reynold Xin 527fc5d0c9 [SPARK-20576][SQL] Support generic hint function in Dataset/DataFrame
## What changes were proposed in this pull request?
We allow users to specify hints (currently only "broadcast" is supported) in SQL and DataFrame. However, while SQL has a standard hint format (/*+ ... */), DataFrame doesn't have one and sometimes users are confused that they can't find how to apply a broadcast hint. This ticket adds a generic hint function on DataFrame that allows using the same hint on DataFrames as well as SQL.

As an example, after this patch, the following will apply a broadcast hint on a DataFrame using the new hint function:

```
df1.join(df2.hint("broadcast"))
```

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

Author: Reynold Xin <rxin@databricks.com>

Closes #17839 from rxin/SPARK-20576.
2017-05-03 09:22:25 -07:00
Sean Owen 16fab6b0ef [SPARK-20523][BUILD] Clean up build warnings for 2.2.0 release
## What changes were proposed in this pull request?

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

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

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

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

## How was this patch tested?

Regression test

Author: Burak Yavuz <brkyvz@gmail.com>

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

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

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

## How was this patch tested?

Tested using unit tests, integration tests, manual tests.

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

Closes #17764 from ptkool/is_not_distinct_from.
2017-05-01 17:05:35 -07:00
Herman van Hovell 6b44c4d63a [SPARK-20534][SQL] Make outer generate exec return empty rows
## What changes were proposed in this pull request?
Generate exec does not produce `null` values if the generator for the input row is empty and the generate operates in outer mode without join. This is caused by the fact that the `join=false` code path is different from the `join=true` code path, and that the `join=false` code path did deal with outer properly. This PR addresses this issue.

## How was this patch tested?
Updated `outer*` tests in `GeneratorFunctionSuite`.

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

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

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

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

**Before**

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

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

**After**

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

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

## How was this patch tested?

Unit tests in `DataTypeParserSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17784 from HyukjinKwon/SPARK-20492.
2017-04-30 08:24:10 -07:00
hyukjinkwon d228cd0b02 [SPARK-20442][PYTHON][DOCS] Fill up documentations for functions in Column API in PySpark
## What changes were proposed in this pull request?

This PR proposes to fill up the documentation with examples for `bitwiseOR`, `bitwiseAND`, `bitwiseXOR`. `contains`, `asc` and `desc` in `Column` API.

Also, this PR fixes minor typos in the documentation and matches some of the contents between Scala doc and Python doc.

Lastly, this PR suggests to use `spark` rather than `sc` in doc tests in `Column` for Python documentation.

## How was this patch tested?

Doc tests were added and manually tested with the commands below:

`./python/run-tests.py --module pyspark-sql`
`./python/run-tests.py --module pyspark-sql --python-executable python3`
`./dev/lint-python`

Output was checked via `make html` under `./python/docs`. The snapshots will be left on the codes with comments.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17737 from HyukjinKwon/SPARK-20442.
2017-04-29 13:46:40 -07:00
Tejas Patil 814a61a867 [SPARK-20487][SQL] Display serde for HiveTableScan node in explained plan
## What changes were proposed in this pull request?

This was a suggestion by rxin at https://github.com/apache/spark/pull/17780#issuecomment-298073408

## How was this patch tested?

- modified existing unit test
- manual testing:

```
scala> hc.sql(" SELECT * FROM tejasp_bucketed_partitioned_1  where name = ''  ").explain(true)
== Parsed Logical Plan ==
'Project [*]
+- 'Filter ('name = )
   +- 'UnresolvedRelation `tejasp_bucketed_partitioned_1`

== Analyzed Logical Plan ==
user_id: bigint, name: string, ds: string
Project [user_id#24L, name#25, ds#26]
+- Filter (name#25 = )
   +- SubqueryAlias tejasp_bucketed_partitioned_1
      +- CatalogRelation `default`.`tejasp_bucketed_partitioned_1`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [user_id#24L, name#25], [ds#26]

== Optimized Logical Plan ==
Filter (isnotnull(name#25) && (name#25 = ))
+- CatalogRelation `default`.`tejasp_bucketed_partitioned_1`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [user_id#24L, name#25], [ds#26]

== Physical Plan ==
*Filter (isnotnull(name#25) && (name#25 = ))
+- HiveTableScan [user_id#24L, name#25, ds#26], CatalogRelation `default`.`tejasp_bucketed_partitioned_1`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [user_id#24L, name#25], [ds#26]
```

Author: Tejas Patil <tejasp@fb.com>

Closes #17806 from tejasapatil/add_serde.
2017-04-28 23:12:26 -07:00
Takeshi Yamamuro 59e3a56444 [SPARK-14471][SQL] Aliases in SELECT could be used in GROUP BY
## What changes were proposed in this pull request?
This pr added a new rule in `Analyzer` to resolve aliases in `GROUP BY`.
The current master throws an exception if `GROUP BY` clauses have aliases in `SELECT`;
```
scala> spark.sql("select a a1, a1 + 1 as b, count(1) from t group by a1")
org.apache.spark.sql.AnalysisException: cannot resolve '`a1`' given input columns: [a]; line 1 pos 51;
'Aggregate ['a1], [a#83L AS a1#87L, ('a1 + 1) AS b#88, count(1) AS count(1)#90L]
+- SubqueryAlias t
   +- Project [id#80L AS a#83L]
      +- Range (0, 10, step=1, splits=Some(8))

  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:77)
  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:289)
```

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17191 from maropu/SPARK-14471.
2017-04-28 14:41:53 +08:00
Tejas Patil a4aa4665a6 [SPARK-20487][SQL] HiveTableScan node is quite verbose in explained plan
## What changes were proposed in this pull request?

Changed `TreeNode.argString` to handle `CatalogTable` separately (otherwise it would call the default `toString` on the `CatalogTable`)

## How was this patch tested?

- Expanded scope of existing unit test to ensure that verbose information is not present
- Manual testing

Before

```
scala> hc.sql(" SELECT * FROM my_table WHERE name = 'foo' ").explain(true)
== Parsed Logical Plan ==
'Project [*]
+- 'Filter ('name = foo)
   +- 'UnresolvedRelation `my_table`

== Analyzed Logical Plan ==
user_id: bigint, name: string, ds: string
Project [user_id#13L, name#14, ds#15]
+- Filter (name#14 = foo)
   +- SubqueryAlias my_table
      +- CatalogRelation CatalogTable(
Database: default
Table: my_table
Owner: tejasp
Created: Fri Apr 14 17:05:50 PDT 2017
Last Access: Wed Dec 31 16:00:00 PST 1969
Type: MANAGED
Provider: hive
Properties: [serialization.format=1]
Statistics: 9223372036854775807 bytes
Location: file:/tmp/warehouse/my_table
Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
InputFormat: org.apache.hadoop.mapred.TextInputFormat
OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
Partition Provider: Catalog
Partition Columns: [`ds`]
Schema: root
-- user_id: long (nullable = true)
-- name: string (nullable = true)
-- ds: string (nullable = true)
), [user_id#13L, name#14], [ds#15]

== Optimized Logical Plan ==
Filter (isnotnull(name#14) && (name#14 = foo))
+- CatalogRelation CatalogTable(
Database: default
Table: my_table
Owner: tejasp
Created: Fri Apr 14 17:05:50 PDT 2017
Last Access: Wed Dec 31 16:00:00 PST 1969
Type: MANAGED
Provider: hive
Properties: [serialization.format=1]
Statistics: 9223372036854775807 bytes
Location: file:/tmp/warehouse/my_table
Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
InputFormat: org.apache.hadoop.mapred.TextInputFormat
OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
Partition Provider: Catalog
Partition Columns: [`ds`]
Schema: root
-- user_id: long (nullable = true)
-- name: string (nullable = true)
-- ds: string (nullable = true)
), [user_id#13L, name#14], [ds#15]

== Physical Plan ==
*Filter (isnotnull(name#14) && (name#14 = foo))
+- HiveTableScan [user_id#13L, name#14, ds#15], CatalogRelation CatalogTable(
Database: default
Table: my_table
Owner: tejasp
Created: Fri Apr 14 17:05:50 PDT 2017
Last Access: Wed Dec 31 16:00:00 PST 1969
Type: MANAGED
Provider: hive
Properties: [serialization.format=1]
Statistics: 9223372036854775807 bytes
Location: file:/tmp/warehouse/my_table
Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
InputFormat: org.apache.hadoop.mapred.TextInputFormat
OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
Partition Provider: Catalog
Partition Columns: [`ds`]
Schema: root
-- user_id: long (nullable = true)
-- name: string (nullable = true)
-- ds: string (nullable = true)
), [user_id#13L, name#14], [ds#15]
```

After

```
scala> hc.sql(" SELECT * FROM my_table WHERE name = 'foo' ").explain(true)
== Parsed Logical Plan ==
'Project [*]
+- 'Filter ('name = foo)
   +- 'UnresolvedRelation `my_table`

== Analyzed Logical Plan ==
user_id: bigint, name: string, ds: string
Project [user_id#13L, name#14, ds#15]
+- Filter (name#14 = foo)
   +- SubqueryAlias my_table
      +- CatalogRelation `default`.`my_table`, [user_id#13L, name#14], [ds#15]

== Optimized Logical Plan ==
Filter (isnotnull(name#14) && (name#14 = foo))
+- CatalogRelation `default`.`my_table`, [user_id#13L, name#14], [ds#15]

== Physical Plan ==
*Filter (isnotnull(name#14) && (name#14 = foo))
+- HiveTableScan [user_id#13L, name#14, ds#15], CatalogRelation `default`.`my_table`, [user_id#13L, name#14], [ds#15]
```

Author: Tejas Patil <tejasp@fb.com>

Closes #17780 from tejasapatil/SPARK-20487_verbose_plan.
2017-04-27 12:13:16 -07:00
Kris Mok 26ac2ce05c [SPARK-20482][SQL] Resolving Casts is too strict on having time zone set
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

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

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

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

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

    public JsonParser createParser(String content)

## How was this patch tested?

existing unit tests

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

Author: Eric Wasserman <ericw@sgn.com>

Closes #17693 from ewasserman/SPARK-20314.
2017-04-26 11:42:43 +08:00
Sameer Agarwal caf392025c [SPARK-18127] Add hooks and extension points to Spark
## What changes were proposed in this pull request?

This patch adds support for customizing the spark session by injecting user-defined custom extensions. This allows a user to add custom analyzer rules/checks, optimizer rules, planning strategies or even a customized parser.

## How was this patch tested?

Unit Tests in SparkSessionExtensionSuite

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

Closes #17724 from sameeragarwal/session-extensions.
2017-04-25 17:05:20 -07:00
Josh Rosen f44c8a843c [SPARK-20453] Bump master branch version to 2.3.0-SNAPSHOT
This patch bumps the master branch version to `2.3.0-SNAPSHOT`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #17753 from JoshRosen/SPARK-20453.
2017-04-24 21:48:04 -07:00
Kazuaki Ishizaki a750a59597 [SPARK-20341][SQL] Support BigInt's value that does not fit in long value range
## What changes were proposed in this pull request?

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

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

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

## How was this patch tested?

Add new test suite into `DecimalSuite`

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

Closes #17684 from kiszk/SPARK-20341.
2017-04-21 22:25:35 +08:00
Juliusz Sompolski c9e6035e1f [SPARK-20412] Throw ParseException from visitNonOptionalPartitionSpec instead of returning null values.
## What changes were proposed in this pull request?

If a partitionSpec is supposed to not contain optional values, a ParseException should be thrown, and not nulls returned.
The nulls can later cause NullPointerExceptions in places not expecting them.

## How was this patch tested?

A query like "SHOW PARTITIONS tbl PARTITION(col1='val1', col2)" used to throw a NullPointerException.
Now it throws a ParseException.

Author: Juliusz Sompolski <julek@databricks.com>

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

The following events are fired per object:

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

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

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

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

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

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

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

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

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

Closes #17641 from hvanhovell/SPARK-20329.
2017-04-21 10:06:12 +08:00
Dilip Biswal d95e4d9d6a [SPARK-20334][SQL] Return a better error message when correlated predicates contain aggregate expression that has mixture of outer and local references.
## What changes were proposed in this pull request?
Address a follow up in [comment](https://github.com/apache/spark/pull/16954#discussion_r105718880)
Currently subqueries with correlated predicates containing aggregate expression having mixture of outer references and local references generate a codegen error like following :

```SQL
SELECT t1a
FROM   t1
GROUP  BY 1
HAVING EXISTS (SELECT 1
               FROM  t2
               WHERE t2a < min(t1a + t2a));
```
Exception snippet.
```
Cannot evaluate expression: min((input[0, int, false] + input[4, int, false]))
	at org.apache.spark.sql.catalyst.expressions.Unevaluable$class.doGenCode(Expression.scala:226)
	at org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression.doGenCode(interfaces.scala:87)
	at org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$genCode$2.apply(Expression.scala:106)
	at org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$genCode$2.apply(Expression.scala:103)
	at scala.Option.getOrElse(Option.scala:121)
	at org.apache.spark.sql.catalyst.expressions.Expression.genCode(Expression.scala:103)

```
After this PR, a better error message is issued.
```
org.apache.spark.sql.AnalysisException
Error in query: Found an aggregate expression in a correlated
predicate that has both outer and local references, which is not supported yet.
Aggregate expression: min((t1.`t1a` + t2.`t2a`)),
Outer references: t1.`t1a`,
Local references: t2.`t2a`.;
```
## How was this patch tested?
Added tests in SQLQueryTestSuite.

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

Closes #17636 from dilipbiswal/subquery_followup1.
2017-04-20 22:35:48 +02:00
Wenchen Fan b91873db09 [SPARK-20409][SQL] fail early if aggregate function in GROUP BY
## What changes were proposed in this pull request?

It's illegal to have aggregate function in GROUP BY, and we should fail at analysis phase, if this happens.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17704 from cloud-fan/minor.
2017-04-20 16:59:38 +02:00
Xiao Li 55bea56911 [SPARK-20156][SQL][FOLLOW-UP] Java String toLowerCase "Turkish locale bug" in Database and Table DDLs
### What changes were proposed in this pull request?
Database and Table names conform the Hive standard ("[a-zA-z_0-9]+"), i.e. if this name only contains characters, numbers, and _.

When calling `toLowerCase` on the names, we should add `Locale.ROOT` to the `toLowerCase`for avoiding inadvertent locale-sensitive variation in behavior (aka the "Turkish locale problem").

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17655 from gatorsmile/locale.
2017-04-20 11:13:48 +01:00
Shixiong Zhu 39e303a8b6 [MINOR][SS] Fix a missing space in UnsupportedOperationChecker error message
## What changes were proposed in this pull request?

Also went through the same file to ensure other string concatenation are correct.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #17691 from zsxwing/fix-error-message.
2017-04-19 18:58:14 -07:00
ptkool 63824b2c8e [SPARK-20350] Add optimization rules to apply Complementation Laws.
## What changes were proposed in this pull request?

Apply Complementation Laws during boolean expression simplification.

## How was this patch tested?

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

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

Closes #17650 from ptkool/apply_complementation_laws.
2017-04-20 09:51:13 +08:00
Koert Kuipers 608bf30f0b [SPARK-20359][SQL] Avoid unnecessary execution in EliminateOuterJoin optimization that can lead to NPE
Avoid necessary execution that can lead to NPE in EliminateOuterJoin and add test in DataFrameSuite to confirm NPE is no longer thrown

## What changes were proposed in this pull request?
Change leftHasNonNullPredicate and rightHasNonNullPredicate to lazy so they are only executed when needed.

## How was this patch tested?

Added test in DataFrameSuite that failed before this fix and now succeeds. Note that a test in catalyst project would be better but i am unsure how to do this.

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

Author: Koert Kuipers <koert@tresata.com>

Closes #17660 from koertkuipers/feat-catch-npe-in-eliminate-outer-join.
2017-04-19 15:52:47 +08:00
Kazuaki Ishizaki e468a96c40 [SPARK-20254][SQL] Remove unnecessary data conversion for Dataset with primitive array
## What changes were proposed in this pull request?

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

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

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

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

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

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

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

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

## How was this patch tested?

Add test suites into `DatasetPrimitiveSuite`

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

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

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

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

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

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

## How was this patch tested?

Add a corresponding test case.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17668 from wzhfy/recursiveReorder.
2017-04-18 20:12:21 +08:00
Felix Cheung b0a1e93e93 [SPARK-17647][SQL][FOLLOWUP][MINOR] fix typo
## What changes were proposed in this pull request?

fix typo

## How was this patch tested?

manual

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #17663 from felixcheung/likedoctypo.
2017-04-17 23:55:40 -07:00
Jacek Laskowski 33ea908af9 [TEST][MINOR] Replace repartitionBy with distribute in CollapseRepartitionSuite
## What changes were proposed in this pull request?

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

## How was this patch tested?

local build and `catalyst/testOnly *CollapseRepartitionSuite`

Author: Jacek Laskowski <jacek@japila.pl>

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

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

---

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

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

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

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

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

Extra case in regex unit tests.

Author: Jakob Odersky <jakob@odersky.com>

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

Closes #15398 from jodersky/SPARK-17647.
2017-04-17 11:17:57 -07:00
Xiao Li 01ff0350a8 [SPARK-20349][SQL] ListFunctions returns duplicate functions after using persistent functions
### What changes were proposed in this pull request?
The session catalog caches some persistent functions in the `FunctionRegistry`, so there can be duplicates. Our Catalog API `listFunctions` does not handle it.

It would be better if `SessionCatalog` API can de-duplciate the records, instead of doing it by each API caller. In `FunctionRegistry`, our functions are identified by the unquoted string. Thus, this PR is try to parse it using our parser interface and then de-duplicate the names.

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17646 from gatorsmile/showFunctions.
2017-04-17 09:50:20 -07:00
Wenchen Fan 35e5ae4f81 [SPARK-19716][SQL][FOLLOW-UP] UnresolvedMapObjects should always be serializable
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/17398 we introduced `UnresolvedMapObjects` as a placeholder of `MapObjects`. Unfortunately `UnresolvedMapObjects` is not serializable as its `function` may reference Scala `Type` which is not serializable.

Ideally this is fine, as we will never serialize and send unresolved expressions to executors. However users may accidentally do this, e.g. mistakenly reference an encoder instance when implementing `Aggregator`, we should fix it so that it's just a performance issue(more network traffic) and should not fail the query.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17639 from cloud-fan/minor.
2017-04-16 11:14:18 +08:00
wangzhenhua fb036c4413 [SPARK-20318][SQL] Use Catalyst type for min/max in ColumnStat for ease of estimation
## What changes were proposed in this pull request?

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

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

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

## How was this patch tested?

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

Author: wangzhenhua <wangzhenhua@huawei.com>

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

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

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

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

## How was this patch tested?

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

Author: Ioana Delaney <ioanamdelaney@gmail.com>

Closes #17546 from ioana-delaney/starSchemaCBOv3.
2017-04-13 22:27:04 +08:00
Reynold Xin 540855382c [SPARK-20304][SQL] AssertNotNull should not include path in string representation
## What changes were proposed in this pull request?
AssertNotNull's toString/simpleString dumps the entire walkedTypePath. walkedTypePath is used for error message reporting and shouldn't be part of the output.

## How was this patch tested?
Manually tested.

Author: Reynold Xin <rxin@databricks.com>

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

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

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

Author: Xiao Li <gatorsmile@gmail.com>

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

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

There are several problems with it:

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

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

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

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

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

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

## How was this patch tested?

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

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

```diff
     profiles_and_goals = build_profiles + sbt_goals

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

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

produces

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

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17477 from HyukjinKwon/SPARK-18692.
2017-04-12 12:38:48 +01:00
jtoka 2e1fd46e12 [SPARK-20296][TRIVIAL][DOCS] Count distinct error message for streaming
## What changes were proposed in this pull request?
Update count distinct error message for streaming datasets/dataframes to match current behavior. These aggregations are not yet supported, regardless of whether the dataset/dataframe is aggregated.

Author: jtoka <jason.tokayer@gmail.com>

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

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

Author: Reynold Xin <rxin@databricks.com>

Closes #17614 from rxin/SPARK-20302.
2017-04-12 01:30:00 -07:00
Dilip Biswal b14bfc3f8e [SPARK-19993][SQL] Caching logical plans containing subquery expressions does not work.
## What changes were proposed in this pull request?
The sameResult() method does not work when the logical plan contains subquery expressions.

**Before the fix**
```SQL
scala> val ds = spark.sql("select * from s1 where s1.c1 in (select s2.c1 from s2 where s1.c1 = s2.c1)")
ds: org.apache.spark.sql.DataFrame = [c1: int]

scala> ds.cache
res13: ds.type = [c1: int]

scala> spark.sql("select * from s1 where s1.c1 in (select s2.c1 from s2 where s1.c1 = s2.c1)").explain(true)
== Analyzed Logical Plan ==
c1: int
Project [c1#86]
+- Filter c1#86 IN (list#78 [c1#86])
   :  +- Project [c1#87]
   :     +- Filter (outer(c1#86) = c1#87)
   :        +- SubqueryAlias s2
   :           +- Relation[c1#87] parquet
   +- SubqueryAlias s1
      +- Relation[c1#86] parquet

== Optimized Logical Plan ==
Join LeftSemi, ((c1#86 = c1#87) && (c1#86 = c1#87))
:- Relation[c1#86] parquet
+- Relation[c1#87] parquet
```
**Plan after fix**
```SQL
== Analyzed Logical Plan ==
c1: int
Project [c1#22]
+- Filter c1#22 IN (list#14 [c1#22])
   :  +- Project [c1#23]
   :     +- Filter (outer(c1#22) = c1#23)
   :        +- SubqueryAlias s2
   :           +- Relation[c1#23] parquet
   +- SubqueryAlias s1
      +- Relation[c1#22] parquet

== Optimized Logical Plan ==
InMemoryRelation [c1#22], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)
   +- *BroadcastHashJoin [c1#1, c1#1], [c1#2, c1#2], LeftSemi, BuildRight
      :- *FileScan parquet default.s1[c1#1] Batched: true, Format: Parquet, Location: InMemoryFileIndex[file:/Users/dbiswal/mygit/apache/spark/bin/spark-warehouse/s1], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<c1:int>
      +- BroadcastExchange HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))))
         +- *FileScan parquet default.s2[c1#2] Batched: true, Format: Parquet, Location: InMemoryFileIndex[file:/Users/dbiswal/mygit/apache/spark/bin/spark-warehouse/s2], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<c1:int>
```
## How was this patch tested?
New tests are added to CachedTableSuite.

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

Closes #17330 from dilipbiswal/subquery_cache_final.
2017-04-12 12:18:01 +08:00
DB Tsai 8ad63ee158 [SPARK-20291][SQL] NaNvl(FloatType, NullType) should not be cast to NaNvl(DoubleType, DoubleType)
## What changes were proposed in this pull request?

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

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

By adding extra rule in TypeCoercion can resolve this issue.

## How was this patch tested?

unite tests.

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

Author: DB Tsai <dbt@netflix.com>

Closes #17606 from dbtsai/fixNaNvl.
2017-04-12 11:19:20 +08:00
Reynold Xin 123b4fbbc3 [SPARK-20289][SQL] Use StaticInvoke to box primitive types
## What changes were proposed in this pull request?
Dataset typed API currently uses NewInstance to box primitive types (i.e. calling the constructor). Instead, it'd be slightly more idiomatic in Java to use PrimitiveType.valueOf, which can be invoked using StaticInvoke expression.

## How was this patch tested?
The change should be covered by existing tests for Dataset encoders.

Author: Reynold Xin <rxin@databricks.com>

Closes #17604 from rxin/SPARK-20289.
2017-04-11 11:12:31 -07:00
Liang-Chi Hsieh cd91f96714 [SPARK-20175][SQL] Exists should not be evaluated in Join operator
## What changes were proposed in this pull request?

Similar to `ListQuery`, `Exists` should not be evaluated in `Join` operator too.

## How was this patch tested?

Jenkins tests.

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

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

Closes #17491 from viirya/dont-push-exists-to-join.
2017-04-11 20:33:10 +08:00
Wenchen Fan c8706980ae [SPARK-20274][SQL] support compatible array element type in encoder
## What changes were proposed in this pull request?

This is a regression caused by SPARK-19716.

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

## How was this patch tested?

new regression tests

Author: Wenchen Fan <wenchen@databricks.com>

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

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

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

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

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

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

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

Author: Xiao Li <gatorsmile@gmail.com>

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

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

**Before**

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

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

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

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

**After**

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

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

## How was this patch tested?

Unit tests in `ExpressionParserSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17566 from HyukjinKwon/SPARK-19518.
2017-04-10 17:45:27 +02:00
Wenchen Fan 3d7f201f2a [SPARK-20229][SQL] add semanticHash to QueryPlan
## What changes were proposed in this pull request?

Like `Expression`, `QueryPlan` should also have a `semanticHash` method, then we can put plans to a hash map and look it up fast. This PR refactors `QueryPlan` to follow `Expression` and put all the normalization logic in `QueryPlan.canonicalized`, so that it's very natural to implement `semanticHash`.

follow-up: improve `CacheManager` to leverage this `semanticHash` and speed up plan lookup, instead of iterating all cached plans.

## How was this patch tested?

existing tests. Note that we don't need to test the `semanticHash` method, once the existing tests prove `sameResult` is correct, we are good.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17541 from cloud-fan/plan-semantic.
2017-04-10 13:36:08 +08:00
Kazuaki Ishizaki 7a63f5e827 [SPARK-20253][SQL] Remove unnecessary nullchecks of a return value from Spark runtime routines in generated Java code
## What changes were proposed in this pull request?

This PR elminates unnecessary nullchecks of a return value from known Spark runtime routines. We know whether a given Spark runtime routine returns ``null`` or not (e.g. ``ArrayData.toDoubleArray()`` never returns ``null``). Thus, we can eliminate a null check for the return value from the Spark runtime routine.

When we run the following example program, now we get the Java code "Without this PR". In this code, since we know ``ArrayData.toDoubleArray()`` never returns ``null```, we can eliminate null checks at lines 90-92, and 97.

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

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

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

## How was this patch tested?

Add test suites to ``DatasetPrimitiveSuite``

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

Closes #17569 from kiszk/SPARK-20253.
2017-04-10 10:47:17 +08:00
Reynold Xin e1afc4dcca [SPARK-20262][SQL] AssertNotNull should throw NullPointerException
## What changes were proposed in this pull request?
AssertNotNull currently throws RuntimeException. It should throw NullPointerException, which is more specific.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #17573 from rxin/SPARK-20262.
2017-04-07 21:14:50 -07:00
Wenchen Fan 7577e9c356 [SPARK-20246][SQL] should not push predicate down through aggregate with non-deterministic expressions
## What changes were proposed in this pull request?

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

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17562 from cloud-fan/filter.
2017-04-07 20:54:18 -07:00
Wenchen Fan ad3cc1312d [SPARK-20245][SQL][MINOR] pass output to LogicalRelation directly
## What changes were proposed in this pull request?

Currently `LogicalRelation` has a `expectedOutputAttributes` parameter, which makes it hard to reason about what the actual output is. Like other leaf nodes, `LogicalRelation` should also take `output` as a parameter, to simplify the logic

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17552 from cloud-fan/minor.
2017-04-07 15:58:50 +08:00
Reynold Xin 626b4cafce [SPARK-19495][SQL] Make SQLConf slightly more extensible - addendum
## What changes were proposed in this pull request?
This is a tiny addendum to SPARK-19495 to remove the private visibility for copy, which is the only package private method in the entire file.

## How was this patch tested?
N/A - no semantic change.

Author: Reynold Xin <rxin@databricks.com>

Closes #17555 from rxin/SPARK-19495-2.
2017-04-06 19:24:03 -07:00
Ioana Delaney 4000f128b7 [SPARK-20231][SQL] Refactor star schema code for the subsequent star join detection in CBO
## What changes were proposed in this pull request?

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

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

Author: Ioana Delaney <ioanamdelaney@gmail.com>

Closes #17544 from ioana-delaney/starSchemaCBOv2.
2017-04-05 18:02:53 -07:00
Dilip Biswal 9d68c67235 [SPARK-20204][SQL][FOLLOWUP] SQLConf should react to change in default timezone settings
## What changes were proposed in this pull request?
Make sure SESSION_LOCAL_TIMEZONE reflects the change in JVM's default timezone setting. Currently several timezone related tests fail as the change to default timezone is not picked up by SQLConf.

## How was this patch tested?
Added an unit test in ConfigEntrySuite

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

Closes #17537 from dilipbiswal/timezone_debug.
2017-04-06 08:33:14 +08:00
Wenchen Fan 295747e597 [SPARK-19716][SQL] support by-name resolution for struct type elements in array
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

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

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

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17521 from cloud-fan/conf.
2017-04-04 11:56:21 -07:00
Xiao Li 26e7bca229 [SPARK-20198][SQL] Remove the inconsistency in table/function name conventions in SparkSession.Catalog APIs
### What changes were proposed in this pull request?
Observed by felixcheung , in `SparkSession`.`Catalog` APIs, we have different conventions/rules for table/function identifiers/names. Most APIs accept the qualified name (i.e., `databaseName`.`tableName` or `databaseName`.`functionName`). However, the following five APIs do not accept it.
- def listColumns(tableName: String): Dataset[Column]
- def getTable(tableName: String): Table
- def getFunction(functionName: String): Function
- def tableExists(tableName: String): Boolean
- def functionExists(functionName: String): Boolean

To make them consistent with the other Catalog APIs, this PR does the changes, updates the function/API comments and adds the `params` to clarify the inputs we allow.

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17518 from gatorsmile/tableIdentifier.
2017-04-04 18:57:46 +08:00
Xiao Li 51d3c854c5 [SPARK-20067][SQL] Unify and Clean Up Desc Commands Using Catalog Interface
### What changes were proposed in this pull request?

This PR is to unify and clean up the outputs of `DESC EXTENDED/FORMATTED` and `SHOW TABLE EXTENDED` by moving the logics into the Catalog interface. The output formats are improved. We also add the missing attributes. It impacts the DDL commands like `SHOW TABLE EXTENDED`, `DESC EXTENDED` and `DESC FORMATTED`.

In addition, by following what we did in Dataset API `printSchema`, we can use `treeString` to show the schema in the more readable way.

Below is the current way:
```
Schema: STRUCT<`a`: STRING (nullable = true), `b`: INT (nullable = true), `c`: STRING (nullable = true), `d`: STRING (nullable = true)>
```
After the change, it should look like
```
Schema: root
 |-- a: string (nullable = true)
 |-- b: integer (nullable = true)
 |-- c: string (nullable = true)
 |-- d: string (nullable = true)
```

### How was this patch tested?
`describe.sql` and `show-tables.sql`

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17394 from gatorsmile/descFollowUp.
2017-04-03 23:30:12 -07:00
Dilip Biswal 3bfb639cb7 [SPARK-10364][SQL] Support Parquet logical type TIMESTAMP_MILLIS
## What changes were proposed in this pull request?

**Description** from JIRA

The TimestampType in Spark SQL is of microsecond precision. Ideally, we should convert Spark SQL timestamp values into Parquet TIMESTAMP_MICROS. But unfortunately parquet-mr hasn't supported it yet.
For the read path, we should be able to read TIMESTAMP_MILLIS Parquet values and pad a 0 microsecond part to read values.
For the write path, currently we are writing timestamps as INT96, similar to Impala and Hive. One alternative is that, we can have a separate SQL option to let users be able to write Spark SQL timestamp values as TIMESTAMP_MILLIS. Of course, in this way the microsecond part will be truncated.
## How was this patch tested?

Added new tests in ParquetQuerySuite and ParquetIOSuite

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

Closes #15332 from dilipbiswal/parquet-time-millis.
2017-04-04 09:53:05 +09:00
Ron Hu e7877fd472 [SPARK-19408][SQL] filter estimation on two columns of same table
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

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

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

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

Closes #17415 from ron8hu/filterTwoColumns.
2017-04-03 17:27:12 -07:00
samelamin 58c9e6e77a [SPARK-20145] Fix range case insensitive bug in SQL
## What changes were proposed in this pull request?
Range in SQL should be case insensitive

## How was this patch tested?
unit test

Author: samelamin <hussam.elamin@gmail.com>
Author: samelamin <sam_elamin@discovery.com>

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

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

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

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

Author: Adrian Ionescu <adrian@databricks.com>

Closes #17510 from adrian-ionescu/InMemoryCatalog.
2017-04-03 08:48:49 -07:00
hyukjinkwon cff11fd20e [SPARK-20166][SQL] Use XXX for ISO 8601 timezone instead of ZZ (FastDateFormat specific) in CSV/JSON timeformat options
## What changes were proposed in this pull request?

This PR proposes to use `XXX` format instead of `ZZ`. `ZZ` seems a `FastDateFormat` specific.

`ZZ` supports "ISO 8601 extended format time zones" but it seems `FastDateFormat` specific option.
I misunderstood this is compatible format with `SimpleDateFormat` when this change is introduced.
Please see [SimpleDateFormat documentation]( https://docs.oracle.com/javase/7/docs/api/java/text/SimpleDateFormat.html#iso8601timezone) and [FastDateFormat documentation](https://commons.apache.org/proper/commons-lang/apidocs/org/apache/commons/lang3/time/FastDateFormat.html).

It seems we better replace `ZZ` to `XXX` because they look using the same strategy - [FastDateParser.java#L930](8767cd4f1a/src/main/java/org/apache/commons/lang3/time/FastDateParser.java (L930)), [FastDateParser.java#L932-L951 ](8767cd4f1a/src/main/java/org/apache/commons/lang3/time/FastDateParser.java (L932-L951)) and [FastDateParser.java#L596-L601](8767cd4f1a/src/main/java/org/apache/commons/lang3/time/FastDateParser.java (L596-L601)).

I also checked the codes and manually debugged it for sure. It seems both cases use the same pattern `( Z|(?:[+-]\\d{2}(?::)\\d{2}))`.

_Note that this should be rather a fix about documentation and not the behaviour change because `ZZ` seems invalid date format in `SimpleDateFormat` as documented in `DataFrameReader` and etc, and both `ZZ` and `XXX` look identically working with `FastDateFormat`_

Current documentation is as below:

```
   * <li>`timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSZZ`): sets the string that
   * indicates a timestamp format. Custom date formats follow the formats at
   * `java.text.SimpleDateFormat`. This applies to timestamp type.</li>
```

## How was this patch tested?

Existing tests should cover this. Also, manually tested as below (BTW, I don't think these are worth being added as tests within Spark):

**Parse**

```scala
scala> new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").parse("2017-03-21T00:00:00.000-11:00")
res4: java.util.Date = Tue Mar 21 20:00:00 KST 2017

scala>  new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").parse("2017-03-21T00:00:00.000Z")
res10: java.util.Date = Tue Mar 21 09:00:00 KST 2017

scala> new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSZZ").parse("2017-03-21T00:00:00.000-11:00")
java.text.ParseException: Unparseable date: "2017-03-21T00:00:00.000-11:00"
  at java.text.DateFormat.parse(DateFormat.java:366)
  ... 48 elided
scala>  new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSZZ").parse("2017-03-21T00:00:00.000Z")
java.text.ParseException: Unparseable date: "2017-03-21T00:00:00.000Z"
  at java.text.DateFormat.parse(DateFormat.java:366)
  ... 48 elided
```

```scala
scala> org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").parse("2017-03-21T00:00:00.000-11:00")
res7: java.util.Date = Tue Mar 21 20:00:00 KST 2017

scala> org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").parse("2017-03-21T00:00:00.000Z")
res1: java.util.Date = Tue Mar 21 09:00:00 KST 2017

scala> org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSZZ").parse("2017-03-21T00:00:00.000-11:00")
res8: java.util.Date = Tue Mar 21 20:00:00 KST 2017

scala> org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSZZ").parse("2017-03-21T00:00:00.000Z")
res2: java.util.Date = Tue Mar 21 09:00:00 KST 2017
```

**Format**

```scala
scala> new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").format(new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").parse("2017-03-21T00:00:00.000-11:00"))
res6: String = 2017-03-21T20:00:00.000+09:00
```

```scala
scala> val fd = org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSZZ")
fd: org.apache.commons.lang3.time.FastDateFormat = FastDateFormat[yyyy-MM-dd'T'HH:mm:ss.SSSZZ,ko_KR,Asia/Seoul]

scala> fd.format(fd.parse("2017-03-21T00:00:00.000-11:00"))
res1: String = 2017-03-21T20:00:00.000+09:00

scala> val fd = org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSXXX")
fd: org.apache.commons.lang3.time.FastDateFormat = FastDateFormat[yyyy-MM-dd'T'HH:mm:ss.SSSXXX,ko_KR,Asia/Seoul]

scala> fd.format(fd.parse("2017-03-21T00:00:00.000-11:00"))
res2: String = 2017-03-21T20:00:00.000+09:00
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17489 from HyukjinKwon/SPARK-20166.
2017-04-03 10:07:41 +01:00
hyukjinkwon d40cbb8618 [SPARK-20143][SQL] DataType.fromJson should throw an exception with better message
## What changes were proposed in this pull request?

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

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

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

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

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

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

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

After this PR,

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

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

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

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

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

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

## How was this patch tested?

Unit test added in `DataTypeSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

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

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

## How was this patch tested?

Added a new stats estimation test for `BroadcastHint`.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17504 from wzhfy/broadcastHintEstimation.
2017-04-01 22:19:08 +08:00
Kunal Khamar 254877c2f0 [SPARK-20164][SQL] AnalysisException not tolerant of null query plan.
## What changes were proposed in this pull request?

The query plan in an `AnalysisException` may be `null` when an `AnalysisException` object is serialized and then deserialized, since `plan` is marked `transient`. Or when someone throws an `AnalysisException` with a null query plan (which should not happen).
`def getMessage` is not tolerant of this and throws a `NullPointerException`, leading to loss of information about the original exception.
The fix is to add a `null` check in `getMessage`.

## How was this patch tested?

- Unit test

Author: Kunal Khamar <kkhamar@outlook.com>

Closes #17486 from kunalkhamar/spark-20164.
2017-03-31 09:17:22 -07:00
Wenchen Fan c734fc504a [SPARK-20121][SQL] simplify NullPropagation with NullIntolerant
## What changes were proposed in this pull request?

Instead of iterating all expressions that can return null for null inputs, we can just check `NullIntolerant`.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17450 from cloud-fan/null.
2017-03-30 13:36:36 -07:00
Jacek Laskowski 0197262a35 [DOCS] Docs-only improvements
…adoc

## What changes were proposed in this pull request?

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

## How was this patch tested?

Local build

Author: Jacek Laskowski <jacek@japila.pl>

Closes #17417 from jaceklaskowski/window-expression-scaladoc.
2017-03-30 16:07:27 +01:00
Takuya UESHIN dd2e7d528c [SPARK-19088][SQL] Fix 2.10 build.
## What changes were proposed in this pull request?

Commit 6c70a38 broke the build for scala 2.10. The commit uses some reflections which are not available in Scala 2.10. This PR fixes them.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

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

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

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

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

Author: Xiao Li <gatorsmile@gmail.com>

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

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17406 from maropu/SPARK-20009.
2017-03-29 12:37:49 -07:00
Wenchen Fan d4fac410e0 [SPARK-20125][SQL] Dataset of type option of map does not work
## What changes were proposed in this pull request?

When we build the deserializer expression for map type, we will use `StaticInvoke` to call `ArrayBasedMapData.toScalaMap`, and declare the return type as `scala.collection.immutable.Map`. If the map is inside an Option, we will wrap this `StaticInvoke` with `WrapOption`, which requires the input to be `scala.collect.Map`. Ideally this should be fine, as `scala.collection.immutable.Map` extends `scala.collect.Map`, but our `ObjectType` is too strict about this, this PR fixes it.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17454 from cloud-fan/map.
2017-03-28 11:47:43 -07:00
wangzhenhua 4fcc214d9e [SPARK-20124][SQL] Join reorder should keep the same order of final project attributes
## What changes were proposed in this pull request?

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

## How was this patch tested?

A new test case is added in `JoinReorderSuite`.

Author: wangzhenhua <wangzhenhua@huawei.com>

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

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

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

## How was this patch tested?

Add a new test case in `FilterPushdownSuite`.

Author: wangzhenhua <wangzhenhua@huawei.com>

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

Optimization of arbitrary Scala sequence deserialization introduced by #16240.

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

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

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

Before:

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

After:

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

Benchmark results before:

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

Benchmark results after:

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

## How was this patch tested?

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

Additionally in Spark Shell:

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

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

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

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

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

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

## How was this patch tested?
Existing tests.

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

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

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

## How was this patch tested?

A new test case is added in `FilterEstimationSuite`.

Author: wangzhenhua <wangzhenhua@huawei.com>

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

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

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

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

Closes #17432 from hvanhovell/SPARK-20086.
2017-03-26 22:47:31 +02:00
Wenchen Fan 0b903caef3 [SPARK-19949][SQL][FOLLOW-UP] move FailureSafeParser from catalyst to sql core
## What changes were proposed in this pull request?

The `FailureSafeParser` is only used in sql core, it doesn't make sense to put it in catalyst module.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17408 from cloud-fan/minor.
2017-03-25 11:46:54 -07:00
Liang-Chi Hsieh e011004bed [SPARK-19846][SQL] Add a flag to disable constraint propagation
## What changes were proposed in this pull request?

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

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

### Benchmark

Run the following codes locally.

    import org.apache.spark.ml.{Pipeline, PipelineStage}
    import org.apache.spark.ml.feature.{OneHotEncoder, StringIndexer, VectorAssembler}
    import org.apache.spark.sql.internal.SQLConf

    spark.conf.set(SQLConf.CONSTRAINT_PROPAGATION_ENABLED.key, false)

    val df = (1 to 40).foldLeft(Seq((1, "foo"), (2, "bar"), (3, "baz")).toDF("id", "x0"))((df, i) => df.withColumn(s"x$i", $"x0"))

    val indexers = df.columns.tail.map(c => new StringIndexer()
      .setInputCol(c)
      .setOutputCol(s"${c}_indexed")
      .setHandleInvalid("skip"))

    val encoders = indexers.map(indexer => new OneHotEncoder()
      .setInputCol(indexer.getOutputCol)
      .setOutputCol(s"${indexer.getOutputCol}_encoded")
      .setDropLast(true))

    val stages: Array[PipelineStage] = indexers ++ encoders
    val pipeline = new Pipeline().setStages(stages)

    val startTime = System.nanoTime
    pipeline.fit(df).transform(df).show
    val runningTime = System.nanoTime - startTime

Before this patch: 1786001 ms ~= 30 mins
After this patch: 26392 ms = less than half of a minute

Related PRs: #16998, #16785.

## How was this patch tested?

Jenkins tests.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #17186 from viirya/add-flag-disable-constraint-propagation.
2017-03-25 00:04:51 +01:00
Jacek Laskowski 9299d071f9 [SQL][MINOR] Fix for typo in Analyzer
## What changes were proposed in this pull request?

Fix for typo in Analyzer

## How was this patch tested?

local build

Author: Jacek Laskowski <jacek@japila.pl>

Closes #17409 from jaceklaskowski/analyzer-typo.
2017-03-24 09:56:05 -07:00
Kazuaki Ishizaki bb823ca4b4 [SPARK-19959][SQL] Fix to throw NullPointerException in df[java.lang.Long].collect
## What changes were proposed in this pull request?

This PR fixes `NullPointerException` in the generated code by Catalyst. When we run the following code, we get the following `NullPointerException`. This is because there is no null checks for `inputadapter_value`  while `java.lang.Long inputadapter_value` at Line 30 may have `null`.

This happen when a type of DataFrame is nullable primitive type such as `java.lang.Long` and the wholestage codegen is used. While the physical plan keeps `nullable=true` in `input[0, java.lang.Long, true].longValue`, `BoundReference.doGenCode` ignores `nullable=true`. Thus, nullcheck code will not be generated and `NullPointerException` will occur.

This PR checks the nullability and correctly generates nullcheck if needed.
```java
sparkContext.parallelize(Seq[java.lang.Long](0L, null, 2L), 1).toDF.collect
```

```java
Caused by: java.lang.NullPointerException
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(generated.java:37)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:393)
...
```

Generated code without this PR
```java
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private scala.collection.Iterator inputadapter_input;
/* 009 */   private UnsafeRow serializefromobject_result;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 012 */
/* 013 */   public GeneratedIterator(Object[] references) {
/* 014 */     this.references = references;
/* 015 */   }
/* 016 */
/* 017 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 018 */     partitionIndex = index;
/* 019 */     this.inputs = inputs;
/* 020 */     inputadapter_input = inputs[0];
/* 021 */     serializefromobject_result = new UnsafeRow(1);
/* 022 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 0);
/* 023 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 024 */
/* 025 */   }
/* 026 */
/* 027 */   protected void processNext() throws java.io.IOException {
/* 028 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 029 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 030 */       java.lang.Long inputadapter_value = (java.lang.Long)inputadapter_row.get(0, null);
/* 031 */
/* 032 */       boolean serializefromobject_isNull = true;
/* 033 */       long serializefromobject_value = -1L;
/* 034 */       if (!false) {
/* 035 */         serializefromobject_isNull = false;
/* 036 */         if (!serializefromobject_isNull) {
/* 037 */           serializefromobject_value = inputadapter_value.longValue();
/* 038 */         }
/* 039 */
/* 040 */       }
/* 041 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 042 */
/* 043 */       if (serializefromobject_isNull) {
/* 044 */         serializefromobject_rowWriter.setNullAt(0);
/* 045 */       } else {
/* 046 */         serializefromobject_rowWriter.write(0, serializefromobject_value);
/* 047 */       }
/* 048 */       append(serializefromobject_result);
/* 049 */       if (shouldStop()) return;
/* 050 */     }
/* 051 */   }
/* 052 */ }
```

Generated code with this PR

```java
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private scala.collection.Iterator inputadapter_input;
/* 009 */   private UnsafeRow serializefromobject_result;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 012 */
/* 013 */   public GeneratedIterator(Object[] references) {
/* 014 */     this.references = references;
/* 015 */   }
/* 016 */
/* 017 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 018 */     partitionIndex = index;
/* 019 */     this.inputs = inputs;
/* 020 */     inputadapter_input = inputs[0];
/* 021 */     serializefromobject_result = new UnsafeRow(1);
/* 022 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 0);
/* 023 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 024 */
/* 025 */   }
/* 026 */
/* 027 */   protected void processNext() throws java.io.IOException {
/* 028 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 029 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 030 */       boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 031 */       java.lang.Long inputadapter_value = inputadapter_isNull ? null : ((java.lang.Long)inputadapter_row.get(0, null));
/* 032 */
/* 033 */       boolean serializefromobject_isNull = true;
/* 034 */       long serializefromobject_value = -1L;
/* 035 */       if (!inputadapter_isNull) {
/* 036 */         serializefromobject_isNull = false;
/* 037 */         if (!serializefromobject_isNull) {
/* 038 */           serializefromobject_value = inputadapter_value.longValue();
/* 039 */         }
/* 040 */
/* 041 */       }
/* 042 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 043 */
/* 044 */       if (serializefromobject_isNull) {
/* 045 */         serializefromobject_rowWriter.setNullAt(0);
/* 046 */       } else {
/* 047 */         serializefromobject_rowWriter.write(0, serializefromobject_value);
/* 048 */       }
/* 049 */       append(serializefromobject_result);
/* 050 */       if (shouldStop()) return;
/* 051 */     }
/* 052 */   }
/* 053 */ }
```

## How was this patch tested?

Added new test suites in `DataFrameSuites`

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #17302 from kiszk/SPARK-19959.
2017-03-24 12:57:56 +08:00
Tathagata Das 82b598b963 [SPARK-20057][SS] Renamed KeyedState to GroupState in mapGroupsWithState
## What changes were proposed in this pull request?

Since the state is tied a "group" in the "mapGroupsWithState" operations, its better to call the state "GroupState" instead of a key. This would make it more general if you extends this operation to RelationGroupedDataset and python APIs.

## How was this patch tested?
Existing unit tests.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #17385 from tdas/SPARK-20057.
2017-03-22 12:30:36 -07:00
hyukjinkwon 80fd070389 [SPARK-20018][SQL] Pivot with timestamp and count should not print internal representation
## What changes were proposed in this pull request?

Currently, when we perform count with timestamp types, it prints the internal representation as the column name as below:

```scala
Seq(new java.sql.Timestamp(1)).toDF("a").groupBy("a").pivot("a").count().show()
```

```
+--------------------+----+
|                   a|1000|
+--------------------+----+
|1969-12-31 16:00:...|   1|
+--------------------+----+
```

This PR proposes to use external Scala value instead of the internal representation in the column names as below:

```
+--------------------+-----------------------+
|                   a|1969-12-31 16:00:00.001|
+--------------------+-----------------------+
|1969-12-31 16:00:...|                      1|
+--------------------+-----------------------+
```

## How was this patch tested?

Unit test in `DataFramePivotSuite` and manual tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17348 from HyukjinKwon/SPARK-20018.
2017-03-22 09:58:46 -07:00
hyukjinkwon 465818389a [SPARK-19949][SQL][FOLLOW-UP] Clean up parse modes and update related comments
## What changes were proposed in this pull request?

This PR proposes to make `mode` options in both CSV and JSON to use `cass object` and fix some related comments related previous fix.

Also, this PR modifies some tests related parse modes.

## How was this patch tested?

Modified unit tests in both `CSVSuite.scala` and `JsonSuite.scala`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17377 from HyukjinKwon/SPARK-19949.
2017-03-22 09:52:37 -07:00
Tathagata Das c1e87e384d [SPARK-20030][SS] Event-time-based timeout for MapGroupsWithState
## What changes were proposed in this pull request?

Adding event time based timeout. The user sets the timeout timestamp directly using `KeyedState.setTimeoutTimestamp`. The keys times out when the watermark crosses the timeout timestamp.

## How was this patch tested?
Unit tests

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #17361 from tdas/SPARK-20030.
2017-03-21 21:27:08 -07:00
zhaorongsheng 7dbc162f12 [SPARK-20017][SQL] change the nullability of function 'StringToMap' from 'false' to 'true'
## What changes were proposed in this pull request?

Change the nullability of function `StringToMap` from `false` to `true`.

Author: zhaorongsheng <334362872@qq.com>

Closes #17350 from zhaorongsheng/bug-fix_strToMap_NPE.
2017-03-21 11:30:55 -07:00
Xin Wu 4c0ff5f585 [SPARK-19261][SQL] Alter add columns for Hive serde and some datasource tables
## What changes were proposed in this pull request?
Support` ALTER TABLE ADD COLUMNS (...) `syntax for Hive serde and some datasource tables.
In this PR, we consider a few aspects:

1. View is not supported for `ALTER ADD COLUMNS`

2. Since tables created in SparkSQL with Hive DDL syntax will populate table properties with schema information, we need make sure the consistency of the schema before and after ALTER operation in order for future use.

3. For embedded-schema type of format, such as `parquet`, we need to make sure that the predicate on the newly-added columns can be evaluated properly, or pushed down properly. In case of the data file does not have the columns for the newly-added columns, such predicates should return as if the column values are NULLs.

4. For datasource table, this feature does not support the following:
4.1 TEXT format, since there is only one default column `value` is inferred for text format data.
4.2 ORC format, since SparkSQL native ORC reader does not support the difference between user-specified-schema and inferred schema from ORC files.
4.3 Third party datasource types that implements RelationProvider, including the built-in JDBC format, since different implementations by the vendors may have different ways to dealing with schema.
4.4 Other datasource types, such as `parquet`, `json`, `csv`, `hive` are supported.

5. Column names being added can not be duplicate of any existing data column or partition column names. Case sensitivity is taken into consideration according to the sql configuration.

6. This feature also supports In-Memory catalog, while Hive support is turned off.
## How was this patch tested?
Add new test cases

Author: Xin Wu <xinwu@us.ibm.com>

Closes #16626 from xwu0226/alter_add_columns.
2017-03-21 08:49:54 -07:00
wangzhenhua 14865d7ff7 [SPARK-17080][SQL][FOLLOWUP] Improve documentation, change buildJoin method structure and add a debug log
## What changes were proposed in this pull request?

1. Improve documentation for class `Cost` and `JoinReorderDP` and method `buildJoin()`.
2. Change code structure of `buildJoin()` to make the logic clearer.
3. Add a debug-level log to record information for join reordering, including time cost, the number of items and the number of plans in memo.

## How was this patch tested?

Not related.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17353 from wzhfy/reorderFollow.
2017-03-21 08:44:09 -07:00
Xiao Li d2dcd6792f [SPARK-20024][SQL][TEST-MAVEN] SessionCatalog reset need to set the current database of ExternalCatalog
### What changes were proposed in this pull request?
SessionCatalog API setCurrentDatabase does not set the current database of the underlying ExternalCatalog. Thus, weird errors could come in the test suites after we call reset. We need to fix it.

So far, have not found the direct impact in the other code paths because we expect all the SessionCatalog APIs should always use the current database value we managed, unless some of code paths skip it. Thus, we fix it in the test-only function reset().

### How was this patch tested?
Multiple test case failures are observed in mvn and add a test case in SessionCatalogSuite.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17354 from gatorsmile/useDB.
2017-03-20 22:52:45 -07:00
Wenchen Fan 68d65fae71 [SPARK-19949][SQL] unify bad record handling in CSV and JSON
## What changes were proposed in this pull request?

Currently JSON and CSV have exactly the same logic about handling bad records, this PR tries to abstract it and put it in a upper level to reduce code duplication.

The overall idea is, we make the JSON and CSV parser to throw a BadRecordException, then the upper level, FailureSafeParser, handles bad records according to the parse mode.

Behavior changes:
1. with PERMISSIVE mode, if the number of tokens doesn't match the schema, previously CSV parser will treat it as a legal record and parse as many tokens as possible. After this PR, we treat it as an illegal record, and put the raw record string in a special column, but we still parse as many tokens as possible.
2. all logging is removed as they are not very useful in practice.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>
Author: hyukjinkwon <gurwls223@gmail.com>
Author: Wenchen Fan <cloud0fan@gmail.com>

Closes #17315 from cloud-fan/bad-record2.
2017-03-20 21:43:14 -07:00
Takeshi Yamamuro 0ec1db5475 [SPARK-19980][SQL] Add NULL checks in Bean serializer
## What changes were proposed in this pull request?
A Bean serializer in `ExpressionEncoder`  could change values when Beans having NULL. A concrete example is as follows;
```
scala> :paste
class Outer extends Serializable {
  private var cls: Inner = _
  def setCls(c: Inner): Unit = cls = c
  def getCls(): Inner = cls
}

class Inner extends Serializable {
  private var str: String = _
  def setStr(s: String): Unit = str = str
  def getStr(): String = str
}

scala> Seq("""{"cls":null}""", """{"cls": {"str":null}}""").toDF().write.text("data")
scala> val encoder = Encoders.bean(classOf[Outer])
scala> val schema = encoder.schema
scala> val df = spark.read.schema(schema).json("data").as[Outer](encoder)
scala> df.show
+------+
|   cls|
+------+
|[null]|
|  null|
+------+

scala> df.map(x => x)(encoder).show()
+------+
|   cls|
+------+
|[null]|
|[null]|     // <-- Value changed
+------+
```

This is because the Bean serializer does not have the NULL-check expressions that the serializer of Scala's product types has. Actually, this value change does not happen in Scala's product types;

```
scala> :paste
case class Outer(cls: Inner)
case class Inner(str: String)

scala> val encoder = Encoders.product[Outer]
scala> val schema = encoder.schema
scala> val df = spark.read.schema(schema).json("data").as[Outer](encoder)
scala> df.show
+------+
|   cls|
+------+
|[null]|
|  null|
+------+

scala> df.map(x => x)(encoder).show()
+------+
|   cls|
+------+
|[null]|
|  null|
+------+
```

This pr added the NULL-check expressions in Bean serializer along with the serializer of Scala's product types.

## How was this patch tested?
Added tests in `JavaDatasetSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17347 from maropu/SPARK-19980.
2017-03-21 11:17:34 +08:00
wangzhenhua e9c91badce [SPARK-20010][SQL] Sort information is lost after sort merge join
## What changes were proposed in this pull request?

After sort merge join for inner join, now we only keep left key ordering. However, after inner join, right key has the same value and order as left key. So if we need another smj on right key, we will unnecessarily add a sort which causes additional cost.

As a more complicated example, A join B on A.key = B.key join C on B.key = C.key join D on A.key = D.key. We will unnecessarily add a sort on B.key when join {A, B} and C, and add a sort on A.key when join {A, B, C} and D.

To fix this, we need to propagate all sorted information (equivalent expressions) from bottom up through `outputOrdering` and `SortOrder`.

## How was this patch tested?

Test cases are added.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17339 from wzhfy/sortEnhance.
2017-03-21 10:43:17 +08:00
Zheng RuiFeng 10691d36de [SPARK-19573][SQL] Make NaN/null handling consistent in approxQuantile
## What changes were proposed in this pull request?
update `StatFunctions.multipleApproxQuantiles` to handle NaN/null

## How was this patch tested?
existing tests and added tests

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #16971 from zhengruifeng/quantiles_nan.
2017-03-20 18:25:59 -07:00
Ioana Delaney 8163911594 [SPARK-17791][SQL] Join reordering using star schema detection
## What changes were proposed in this pull request?

Star schema consists of one or more fact tables referencing a number of dimension tables. In general, queries against star schema are expected to run fast because of the established RI constraints among the tables. This design proposes a join reordering based on natural, generally accepted heuristics for star schema queries:
- Finds the star join with the largest fact table and places it on the driving arm of the left-deep join. This plan avoids large tables on the inner, and thus favors hash joins.
- Applies the most selective dimensions early in the plan to reduce the amount of data flow.

The design document was included in SPARK-17791.

Link to the google doc: [StarSchemaDetection](https://docs.google.com/document/d/1UAfwbm_A6wo7goHlVZfYK99pqDMEZUumi7pubJXETEA/edit?usp=sharing)

## How was this patch tested?

A new test suite StarJoinSuite.scala was implemented.

Author: Ioana Delaney <ioanamdelaney@gmail.com>

Closes #15363 from ioana-delaney/starJoinReord2.
2017-03-20 16:04:58 +08:00
hyukjinkwon 0cdcf91145 [SPARK-19849][SQL] Support ArrayType in to_json to produce JSON array
## What changes were proposed in this pull request?

This PR proposes to support an array of struct type in `to_json` as below:

```scala
import org.apache.spark.sql.functions._

val df = Seq(Tuple1(Tuple1(1) :: Nil)).toDF("a")
df.select(to_json($"a").as("json")).show()
```

```
+----------+
|      json|
+----------+
|[{"_1":1}]|
+----------+
```

Currently, it throws an exception as below (a newline manually inserted for readability):

```
org.apache.spark.sql.AnalysisException: cannot resolve 'structtojson(`array`)' due to data type
mismatch: structtojson requires that the expression is a struct expression.;;
```

This allows the roundtrip with `from_json` as below:

```scala
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._

val schema = ArrayType(StructType(StructField("a", IntegerType) :: Nil))
val df = Seq("""[{"a":1}, {"a":2}]""").toDF("json").select(from_json($"json", schema).as("array"))
df.show()

// Read back.
df.select(to_json($"array").as("json")).show()
```

```
+----------+
|     array|
+----------+
|[[1], [2]]|
+----------+

+-----------------+
|             json|
+-----------------+
|[{"a":1},{"a":2}]|
+-----------------+
```

Also, this PR proposes to rename from `StructToJson` to `StructsToJson ` and `JsonToStruct` to `JsonToStructs`.

## How was this patch tested?

Unit tests in `JsonFunctionsSuite` and `JsonExpressionsSuite` for Scala, doctest for Python and test in `test_sparkSQL.R` for R.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17192 from HyukjinKwon/SPARK-19849.
2017-03-19 22:33:01 -07:00
Tathagata Das 990af630d0 [SPARK-19067][SS] Processing-time-based timeout in MapGroupsWithState
## What changes were proposed in this pull request?

When a key does not get any new data in `mapGroupsWithState`, the mapping function is never called on it. So we need a timeout feature that calls the function again in such cases, so that the user can decide whether to continue waiting or clean up (remove state, save stuff externally, etc.).
Timeouts can be either based on processing time or event time. This JIRA is for processing time, but defines the high level API design for both. The usage would look like this.
```
def stateFunction(key: K, value: Iterator[V], state: KeyedState[S]): U = {
  ...
  state.setTimeoutDuration(10000)
  ...
}

dataset					// type is Dataset[T]
  .groupByKey[K](keyingFunc)   // generates KeyValueGroupedDataset[K, T]
  .mapGroupsWithState[S, U](
     func = stateFunction,
     timeout = KeyedStateTimeout.withProcessingTime)	// returns Dataset[U]
```

Note the following design aspects.

- The timeout type is provided as a param in mapGroupsWithState as a parameter global to all the keys. This is so that the planner knows this at planning time, and accordingly optimize the execution based on whether to saves extra info in state or not (e.g. timeout durations or timestamps).

- The exact timeout duration is provided inside the function call so that it can be customized on a per key basis.

- When the timeout occurs for a key, the function is called with no values, and KeyedState.isTimingOut() set to true.

- The timeout is reset for key every time the function is called on the key, that is, when the key has new data, or the key has timed out. So the user has to set the timeout duration everytime the function is called, otherwise there will not be any timeout set.

Guarantees provided on timeout of key, when timeout duration is D ms:
- Timeout will never be called before real clock time has advanced by D ms
- Timeout will be called eventually when there is a trigger with any data in it (i.e. after D ms). So there is a no strict upper bound on when the timeout would occur. For example, if there is no data in the stream (for any key) for a while, then the timeout will not be hit.

Implementation details:
- Added new param to `mapGroupsWithState` for timeout
- Added new method to `StateStore` to filter data based on timeout timestamp
- Changed the internal map type of `HDFSBackedStateStore` from Java's `HashMap` to `ConcurrentHashMap` as the latter allows weakly-consistent fail-safe iterators on the map data. See comments in code for more details.
- Refactored logic of `MapGroupsWithStateExec` to
  - Save timeout info to state store for each key that has data.
  - Then, filter states that should be timed out based on the current batch processing timestamp.
- Moved KeyedState for `o.a.s.sql` to `o.a.s.sql.streaming`. I remember that this was a feedback in the MapGroupsWithState PR that I had forgotten to address.

## How was this patch tested?
New unit tests in
- MapGroupsWithStateSuite for timeouts.
- StateStoreSuite for new APIs in StateStore.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #17179 from tdas/mapgroupwithstate-timeout.
2017-03-19 14:07:49 -07:00
Takeshi Yamamuro ccba622e35 [SPARK-19896][SQL] Throw an exception if case classes have circular references in toDS
## What changes were proposed in this pull request?
If case classes have circular references below, it throws StackOverflowError;
```
scala> :pasge
case class classA(i: Int, cls: classB)
case class classB(cls: classA)

scala> Seq(classA(0, null)).toDS()
java.lang.StackOverflowError
  at scala.reflect.internal.Symbols$Symbol.info(Symbols.scala:1494)
  at scala.reflect.runtime.JavaMirrors$JavaMirror$$anon$1.scala$reflect$runtime$SynchronizedSymbols$SynchronizedSymbol$$super$info(JavaMirrors.scala:66)
  at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$$anonfun$info$1.apply(SynchronizedSymbols.scala:127)
  at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$$anonfun$info$1.apply(SynchronizedSymbols.scala:127)
  at scala.reflect.runtime.Gil$class.gilSynchronized(Gil.scala:19)
  at scala.reflect.runtime.JavaUniverse.gilSynchronized(JavaUniverse.scala:16)
  at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$class.gilSynchronizedIfNotThreadsafe(SynchronizedSymbols.scala:123)
  at scala.reflect.runtime.JavaMirrors$JavaMirror$$anon$1.gilSynchronizedIfNotThreadsafe(JavaMirrors.scala:66)
  at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$class.info(SynchronizedSymbols.scala:127)
  at scala.reflect.runtime.JavaMirrors$JavaMirror$$anon$1.info(JavaMirrors.scala:66)
  at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:48)
  at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:45)
  at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:45)
  at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:45)
  at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:45)
```
This pr added code to throw UnsupportedOperationException in that case as follows;
```
scala> :paste
case class A(cls: B)
case class B(cls: A)

scala> Seq(A(null)).toDS()
java.lang.UnsupportedOperationException: cannot have circular references in class, but got the circular reference of class B
  at org.apache.spark.sql.catalyst.ScalaReflection$.org$apache$spark$sql$catalyst$ScalaReflection$$serializerFor(ScalaReflection.scala:627)
  at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$9.apply(ScalaReflection.scala:644)
  at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$9.apply(ScalaReflection.scala:632)
  at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
  at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
  at scala.collection.immutable.List.foreach(List.scala:381)
  at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241)
```

## How was this patch tested?
Added tests in `DatasetSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17318 from maropu/SPARK-19896.
2017-03-18 14:40:16 +08:00
wangzhenhua c083b6b7de [SPARK-19915][SQL] Exclude cartesian product candidates to reduce the search space
## What changes were proposed in this pull request?

We have some concerns about removing size in the cost model [in the previous pr](https://github.com/apache/spark/pull/17240). It's a tradeoff between code structure and algorithm completeness. I tend to keep the size and thus create this new pr without changing cost model.

What this pr does:
1. We only consider consecutive inner joinable items, thus excluding cartesian products in reordering procedure. This significantly reduces the search space and memory overhead of memo. Otherwise every combination of items will exist in the memo.
2. This pr also includes a bug fix: if a leaf item is a project(_, child), current solution will miss the project.

## How was this patch tested?

Added test cases.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17286 from wzhfy/joinReorder3.
2017-03-18 14:07:25 +08:00
Takeshi Yamamuro 7de66bae58 [SPARK-19967][SQL] Add from_json in FunctionRegistry
## What changes were proposed in this pull request?
This pr added entries in `FunctionRegistry` and supported `from_json` in SQL.

## How was this patch tested?
Added tests in `JsonFunctionsSuite` and `SQLQueryTestSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17320 from maropu/SPARK-19967.
2017-03-17 14:51:59 -07:00
Andrew Ray 13538cf3dd [SPARK-19882][SQL] Pivot with null as a distinct pivot value throws NPE
## What changes were proposed in this pull request?

Allows null values of the pivot column to be included in the pivot values list without throwing NPE

Note this PR was made as an alternative to #17224 but preserves the two phase aggregate operation that is needed for good performance.

## How was this patch tested?

Additional unit test

Author: Andrew Ray <ray.andrew@gmail.com>

Closes #17226 from aray/pivot-null.
2017-03-17 16:43:42 +08:00
windpiger 8e8f898335 [SPARK-19945][SQL] add test suite for SessionCatalog with HiveExternalCatalog
## What changes were proposed in this pull request?

Currently `SessionCatalogSuite` is only for `InMemoryCatalog`, there is no suite for `HiveExternalCatalog`.
And there are some ddl function is not proper to test in `ExternalCatalogSuite`, because some logic are not full implement in `ExternalCatalog`, these ddl functions are full implement in `SessionCatalog`(e.g. merge the same logic from `ExternalCatalog` up to `SessionCatalog` ).
It is better to test it in `SessionCatalogSuite` for this situation.

So we should add a test suite for `SessionCatalog` with `HiveExternalCatalog`

The main change is that in `SessionCatalogSuite` add two functions:
`withBasicCatalog` and `withEmptyCatalog`
And replace the code like  `val catalog = new SessionCatalog(newBasicCatalog)` with above two functions

## How was this patch tested?
add `HiveExternalSessionCatalogSuite`

Author: windpiger <songjun@outlook.com>

Closes #17287 from windpiger/sessioncatalogsuit.
2017-03-16 11:34:13 -07:00
Xiao Li 1472cac4bb [SPARK-19830][SQL] Add parseTableSchema API to ParserInterface
### What changes were proposed in this pull request?

Specifying the table schema in DDL formats is needed for different scenarios. For example,
- [specifying the schema in SQL function `from_json` using DDL formats](https://issues.apache.org/jira/browse/SPARK-19637), which is suggested by marmbrus ,
- [specifying the customized JDBC data types](https://github.com/apache/spark/pull/16209).

These two PRs need users to use the JSON format to specify the table schema. This is not user friendly.

This PR is to provide a `parseTableSchema` API in `ParserInterface`.

### How was this patch tested?
Added a test suite `TableSchemaParserSuite`

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17171 from gatorsmile/parseDDLStmt.
2017-03-16 12:06:20 +08:00
Takeshi Yamamuro 21f333c635 [SPARK-19751][SQL] Throw an exception if bean class has one's own class in fields
## What changes were proposed in this pull request?
The current master throws `StackOverflowError` in `createDataFrame`/`createDataset` if bean has one's own class in fields;
```
public class SelfClassInFieldBean implements Serializable {
  private SelfClassInFieldBean child;
  ...
}
```
This pr added code to throw `UnsupportedOperationException` in that case as soon as possible.

## How was this patch tested?
Added tests in `JavaDataFrameSuite` and `JavaDatasetSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17188 from maropu/SPARK-19751.
2017-03-16 08:50:01 +08:00
windpiger fc9314671c [SPARK-19961][SQL][MINOR] unify a erro msg when drop databse for HiveExternalCatalog and InMemoryCatalog
## What changes were proposed in this pull request?

unify a exception erro msg for dropdatabase when the database still have some tables for HiveExternalCatalog and InMemoryCatalog
## How was this patch tested?
N/A

Author: windpiger <songjun@outlook.com>

Closes #17305 from windpiger/unifyErromsg.
2017-03-16 08:44:57 +08:00
Tejas Patil 02c274eaba [SPARK-13450] Introduce ExternalAppendOnlyUnsafeRowArray. Change CartesianProductExec, SortMergeJoin, WindowExec to use it
## What issue does this PR address ?

Jira: https://issues.apache.org/jira/browse/SPARK-13450

In `SortMergeJoinExec`, rows of the right relation having the same value for a join key are buffered in-memory. In case of skew, this causes OOMs (see comments in SPARK-13450 for more details). Heap dump from a failed job confirms this : https://issues.apache.org/jira/secure/attachment/12846382/heap-dump-analysis.png . While its possible to increase the heap size to workaround, Spark should be resilient to such issues as skews can happen arbitrarily.

## Change proposed in this pull request

- Introduces `ExternalAppendOnlyUnsafeRowArray`
  - It holds `UnsafeRow`s in-memory upto a certain threshold.
  - After the threshold is hit, it switches to `UnsafeExternalSorter` which enables spilling of the rows to disk. It does NOT sort the data.
  - Allows iterating the array multiple times. However, any alteration to the array (using `add` or `clear`) will invalidate the existing iterator(s)
- `WindowExec` was already using `UnsafeExternalSorter` to support spilling. Changed it to use the new array
- Changed `SortMergeJoinExec` to use the new array implementation
  - NOTE: I have not changed FULL OUTER JOIN to use this new array implementation. Changing that will need more surgery and I will rather put up a separate PR for that once this gets in.
- Changed `CartesianProductExec` to use the new array implementation

#### Note for reviewers

The diff can be divided into 3 parts. My motive behind having all the changes in a single PR was to demonstrate that the API is sane and supports 2 use cases. If reviewing as 3 separate PRs would help, I am happy to make the split.

## How was this patch tested ?

#### Unit testing
- Added unit tests `ExternalAppendOnlyUnsafeRowArray` to validate all its APIs and access patterns
- Added unit test for `SortMergeExec`
  - with and without spill for inner join, left outer join, right outer join to confirm that the spill threshold config behaves as expected and output is as expected.
  - This PR touches the scanning logic in `SortMergeExec` for _all_ joins (except FULL OUTER JOIN). However, I expect existing test cases to cover that there is no regression in correctness.
- Added unit test for `WindowExec` to check behavior of spilling and correctness of results.

#### Stress testing
- Confirmed that OOM is gone by running against a production job which used to OOM
- Since I cannot share details about prod workload externally, created synthetic data to mimic the issue. Ran before and after the fix to demonstrate the issue and query success with this PR

Generating the synthetic data

```
./bin/spark-shell --driver-memory=6G

import org.apache.spark.sql._
val hc = SparkSession.builder.master("local").getOrCreate()

hc.sql("DROP TABLE IF EXISTS spark_13450_large_table").collect
hc.sql("DROP TABLE IF EXISTS spark_13450_one_row_table").collect

val df1 = (0 until 1).map(i => ("10", "100", i.toString, (i * 2).toString)).toDF("i", "j", "str1", "str2")
df1.write.format("org.apache.spark.sql.hive.orc.OrcFileFormat").bucketBy(100, "i", "j").sortBy("i", "j").saveAsTable("spark_13450_one_row_table")

val df2 = (0 until 3000000).map(i => ("10", "100", i.toString, (i * 2).toString)).toDF("i", "j", "str1", "str2")
df2.write.format("org.apache.spark.sql.hive.orc.OrcFileFormat").bucketBy(100, "i", "j").sortBy("i", "j").saveAsTable("spark_13450_large_table")
```

Ran this against trunk VS local build with this PR. OOM repros with trunk and with the fix this query runs fine.

```
./bin/spark-shell --driver-java-options="-XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/tmp/spark.driver.heapdump.hprof"

import org.apache.spark.sql._
val hc = SparkSession.builder.master("local").getOrCreate()
hc.sql("SET spark.sql.autoBroadcastJoinThreshold=1")
hc.sql("SET spark.sql.sortMergeJoinExec.buffer.spill.threshold=10000")

hc.sql("DROP TABLE IF EXISTS spark_13450_result").collect
hc.sql("""
  CREATE TABLE spark_13450_result
  AS
  SELECT
    a.i AS a_i, a.j AS a_j, a.str1 AS a_str1, a.str2 AS a_str2,
    b.i AS b_i, b.j AS b_j, b.str1 AS b_str1, b.str2 AS b_str2
  FROM
    spark_13450_one_row_table a
  JOIN
    spark_13450_large_table b
  ON
    a.i=b.i AND
    a.j=b.j
""")
```

## Performance comparison

### Macro-benchmark

I ran a SMB join query over two real world tables (2 trillion rows (40 TB) and 6 million rows (120 GB)). Note that this dataset does not have skew so no spill happened. I saw improvement in CPU time by 2-4% over version without this PR. This did not add up as I was expected some regression. I think allocating array of capacity of 128 at the start (instead of starting with default size 16) is the sole reason for the perf. gain : https://github.com/tejasapatil/spark/blob/SPARK-13450_smb_buffer_oom/sql/core/src/main/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArray.scala#L43 . I could remove that and rerun, but effectively the change will be deployed in this form and I wanted to see the effect of it over large workload.

### Micro-benchmark

Two types of benchmarking can be found in `ExternalAppendOnlyUnsafeRowArrayBenchmark`:

[A] Comparing `ExternalAppendOnlyUnsafeRowArray` against raw `ArrayBuffer` when all rows fit in-memory and there is no spill

```
Array with 1000 rows:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
ArrayBuffer                                   7821 / 7941         33.5          29.8       1.0X
ExternalAppendOnlyUnsafeRowArray              8798 / 8819         29.8          33.6       0.9X

Array with 30000 rows:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
ArrayBuffer                                 19200 / 19206         25.6          39.1       1.0X
ExternalAppendOnlyUnsafeRowArray            19558 / 19562         25.1          39.8       1.0X

Array with 100000 rows:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
ArrayBuffer                                   5949 / 6028         17.2          58.1       1.0X
ExternalAppendOnlyUnsafeRowArray              6078 / 6138         16.8          59.4       1.0X
```

[B] Comparing `ExternalAppendOnlyUnsafeRowArray` against raw `UnsafeExternalSorter` when there is spilling of data

```
Spilling with 1000 rows:                 Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
UnsafeExternalSorter                          9239 / 9470         28.4          35.2       1.0X
ExternalAppendOnlyUnsafeRowArray              8857 / 8909         29.6          33.8       1.0X

Spilling with 10000 rows:                Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
UnsafeExternalSorter                             4 /    5         39.3          25.5       1.0X
ExternalAppendOnlyUnsafeRowArray                 5 /    6         29.8          33.5       0.8X
```

Author: Tejas Patil <tejasp@fb.com>

Closes #16909 from tejasapatil/SPARK-13450_smb_buffer_oom.
2017-03-15 20:18:39 +01:00
jiangxingbo ee36bc1c90 [SPARK-19877][SQL] Restrict the nested level of a view
## What changes were proposed in this pull request?

We should restrict the nested level of a view, to avoid stack overflow exception during the view resolution.

## How was this patch tested?

Add new test case in `SQLViewSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #17241 from jiangxb1987/view-depth.
2017-03-14 23:57:54 -07:00
Wenchen Fan dacc382f0c [SPARK-19887][SQL] dynamic partition keys can be null or empty string
## What changes were proposed in this pull request?

When dynamic partition value is null or empty string, we should write the data to a directory like `a=__HIVE_DEFAULT_PARTITION__`, when we read the data back, we should respect this special directory name and treat it as null.

This is the same behavior of impala, see https://issues.apache.org/jira/browse/IMPALA-252

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17277 from cloud-fan/partition.
2017-03-15 08:24:41 +08:00
Takuya UESHIN 7ded39c223 [SPARK-19817][SQL] Make it clear that timeZone option is a general option in DataFrameReader/Writer.
## What changes were proposed in this pull request?

As timezone setting can also affect partition values, it works for all formats, we should make it clear.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #17281 from ueshin/issues/SPARK-19817.
2017-03-14 13:57:23 -07:00
Nattavut Sutyanyong 6eac96823c [SPARK-18966][SQL] NOT IN subquery with correlated expressions may return incorrect result
## What changes were proposed in this pull request?

This PR fixes the following problem:
````
Seq((1, 2)).toDF("a1", "a2").createOrReplaceTempView("a")
Seq[(java.lang.Integer, java.lang.Integer)]((1, null)).toDF("b1", "b2").createOrReplaceTempView("b")

// The expected result is 1 row of (1,2) as shown in the next statement.
sql("select * from a where a1 not in (select b1 from b where b2 = a2)").show
+---+---+
| a1| a2|
+---+---+
+---+---+

sql("select * from a where a1 not in (select b1 from b where b2 = 2)").show
+---+---+
| a1| a2|
+---+---+
|  1|  2|
+---+---+
````
There are a number of scenarios to consider:

1. When the correlated predicate yields a match (i.e., B.B2 = A.A2)
1.1. When the NOT IN expression yields a match (i.e., A.A1 = B.B1)
1.2. When the NOT IN expression yields no match (i.e., A.A1 = B.B1 returns false)
1.3. When A.A1 is null
1.4. When B.B1 is null
1.4.1. When A.A1 is not null
1.4.2. When A.A1 is null

2. When the correlated predicate yields no match (i.e.,B.B2 = A.A2 is false or unknown)
2.1. When B.B2 is null and A.A2 is null
2.2. When B.B2 is null and A.A2 is not null
2.3. When the value of A.A2 does not match any of B.B2

````
 A.A1   A.A2      B.B1   B.B2
-----  -----     -----  -----
    1      1         1      1    (1.1)
    2      1                     (1.2)
 null      1                     (1.3)

    1      3      null      3    (1.4.1)
 null      3                     (1.4.2)

    1   null         1   null    (2.1)
 null      2                     (2.2 & 2.3)
````

We can divide the evaluation of the above correlated NOT IN subquery into 2 groups:-

Group 1: The rows in A when there is a match from the correlated predicate (A.A1 = B.B1)

In this case, the result of the subquery is not empty and the semantics of the NOT IN depends solely on the evaluation of the equality comparison of the columns of NOT IN, i.e., A1 = B1, which says

- If A.A1 is null, the row is filtered (1.3 and 1.4.2)
- If A.A1 = B.B1, the row is filtered (1.1)
- If B.B1 is null, any rows of A in the same group (A.A2 = B.B2) is filtered (1.4.1 & 1.4.2)
- Otherwise, the row is qualified.

Hence, in this group, the result is the row from (1.2).

Group 2: The rows in A when there is no match from the correlated predicate (A.A2 = B.B2)

In this case, all the rows in A, including the rows where A.A1, are qualified because the subquery returns an empty set and by the semantics of the NOT IN, all rows from the parent side qualifies as the result set, that is, the rows from (2.1, 2.2 and 2.3).

In conclusion, the correct result set of the above query is
````
 A.A1   A.A2
-----  -----
    2      1    (1.2)
    1   null    (2.1)
 null      2    (2.2 & 2.3)
````
## How was this patch tested?
unit tests, regression tests, and new test cases focusing on the problem being fixed.

Author: Nattavut Sutyanyong <nsy.can@gmail.com>

Closes #17294 from nsyca/18966.
2017-03-14 20:34:59 +01:00
Herman van Hovell e04c05cf41 [SPARK-19933][SQL] Do not change output of a subquery
## What changes were proposed in this pull request?
The `RemoveRedundantAlias` rule can change the output attributes (the expression id's to be precise) of a query by eliminating the redundant alias producing them. This is no problem for a regular query, but can cause problems for correlated subqueries: The attributes produced by the subquery are used in the parent plan; changing them will break the parent plan.

This PR fixes this by wrapping a subquery in a `Subquery` top level node when it gets optimized. The `RemoveRedundantAlias` rule now recognizes `Subquery` and makes sure that the output attributes of the `Subquery` node are retained.

## How was this patch tested?
Added a test case to `RemoveRedundantAliasAndProjectSuite` and added a regression test to `SubquerySuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17278 from hvanhovell/SPARK-19933.
2017-03-14 18:52:16 +01:00
Herman van Hovell 1c7275efa7 [SPARK-18874][SQL] Fix 2.10 build after moving the subquery rules to optimization
## What changes were proposed in this pull request?
Commit 4ce970d714 in accidentally broke the 2.10 build for Spark. This PR fixes this by simplifying the offending pattern match.

## How was this patch tested?
Existing tests.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17288 from hvanhovell/SPARK-18874.
2017-03-14 14:02:48 +01:00
Herman van Hovell a0b92f73fe [SPARK-19850][SQL] Allow the use of aliases in SQL function calls
## What changes were proposed in this pull request?
We currently cannot use aliases in SQL function calls. This is inconvenient when you try to create a struct. This SQL query for example `select struct(1, 2) st`, will create a struct with column names `col1` and `col2`. This is even more problematic when we want to append a field to an existing struct. For example if we want to a field to struct `st` we would issue the following SQL query `select struct(st.*, 1) as st from src`, the result will be struct `st` with an a column with a non descriptive name `col3` (if `st` itself has 2 fields).

This PR proposes to change this by allowing the use of aliased expression in function parameters. For example `select struct(1 as a, 2 as b) st`, will create a struct with columns `a` & `b`.

## How was this patch tested?
Added a test to `ExpressionParserSuite` and added a test file for `SQLQueryTestSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17245 from hvanhovell/SPARK-19850.
2017-03-14 12:49:30 +01:00
Reynold Xin 0ee38a39e4 [SPARK-19944][SQL] Move SQLConf from sql/core to sql/catalyst
## What changes were proposed in this pull request?
This patch moves SQLConf from sql/core to sql/catalyst. To minimize the changes, the patch used type alias to still keep CatalystConf (as a type alias) and SimpleCatalystConf (as a concrete class that extends SQLConf).

Motivation for the change is that it is pretty weird to have SQLConf only in sql/core and then we have to duplicate config options that impact optimizer/analyzer in sql/catalyst using CatalystConf.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #17285 from rxin/SPARK-19944.
2017-03-14 19:02:30 +08:00
Nattavut Sutyanyong 4ce970d714 [SPARK-18874][SQL] First phase: Deferring the correlated predicate pull up to Optimizer phase
## What changes were proposed in this pull request?
Currently Analyzer as part of ResolveSubquery, pulls up the correlated predicates to its
originating SubqueryExpression. The subquery plan is then transformed to remove the correlated
predicates after they are moved up to the outer plan. In this PR, the task of pulling up
correlated predicates is deferred to Optimizer. This is the initial work that will allow us to
support the form of correlated subqueries that we don't support today. The design document
from nsyca can be found in the following link :
[DesignDoc](https://docs.google.com/document/d/1QDZ8JwU63RwGFS6KVF54Rjj9ZJyK33d49ZWbjFBaIgU/edit#)

The brief description of code changes (hopefully to aid with code review) can be be found in the
following link:
[CodeChanges](https://docs.google.com/document/d/18mqjhL9V1An-tNta7aVE13HkALRZ5GZ24AATA-Vqqf0/edit#)

## How was this patch tested?
The test case PRs were submitted earlier using.
[16337](https://github.com/apache/spark/pull/16337) [16759](https://github.com/apache/spark/pull/16759) [16841](https://github.com/apache/spark/pull/16841) [16915](https://github.com/apache/spark/pull/16915) [16798](https://github.com/apache/spark/pull/16798) [16712](https://github.com/apache/spark/pull/16712) [16710](https://github.com/apache/spark/pull/16710) [16760](https://github.com/apache/spark/pull/16760) [16802](https://github.com/apache/spark/pull/16802)

Author: Dilip Biswal <dbiswal@us.ibm.com>

Closes #16954 from dilipbiswal/SPARK-18874.
2017-03-14 10:37:10 +01:00
Tejas Patil 9456688547 [SPARK-17495][SQL] Support date, timestamp and interval types in Hive hash
## What changes were proposed in this pull request?

- Timestamp hashing is done as per [TimestampWritable.hashCode()](ff67cdda1c/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritable.java (L406)) in Hive
- Interval hashing is done as per [HiveIntervalDayTime.hashCode()](ff67cdda1c/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveIntervalDayTime.java (L178)). Note that there are inherent differences in how Hive and Spark store intervals under the hood which limits the ability to be in completely sync with hive's hashing function. I have explained this in the method doc.
- Date type was already supported. This PR adds test for that.

## How was this patch tested?

Added unit tests

Author: Tejas Patil <tejasp@fb.com>

Closes #17062 from tejasapatil/SPARK-17495_time_related_types.
2017-03-12 20:08:44 -07:00
Wenchen Fan fb9beda546 [SPARK-19893][SQL] should not run DataFrame set oprations with map type
## What changes were proposed in this pull request?

In spark SQL, map type can't be used in equality test/comparison, and `Intersect`/`Except`/`Distinct` do need equality test for all columns, we should not allow map type in `Intersect`/`Except`/`Distinct`.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17236 from cloud-fan/map.
2017-03-10 16:14:22 -08:00
Kazuaki Ishizaki 5949e6c447 [SPARK-19008][SQL] Improve performance of Dataset.map by eliminating boxing/unboxing
## What changes were proposed in this pull request?

This PR improve performance of Dataset.map() for primitive types by removing boxing/unbox operations. This is based on [the discussion](https://github.com/apache/spark/pull/16391#discussion_r93788919) with cloud-fan.

Current Catalyst generates a method call to a `apply()` method of an anonymous function written in Scala. The types of an argument and return value are `java.lang.Object`. As a result, each method call for a primitive value involves a pair of unboxing and boxing for calling this `apply()` method and a pair of boxing and unboxing for returning from this `apply()` method.

This PR directly calls a specialized version of a `apply()` method without boxing and unboxing. For example, if types of an arguments ant return value is `int`, this PR generates a method call to `apply$mcII$sp`. This PR supports any combination of `Int`, `Long`, `Float`, and `Double`.

The following is a benchmark result using [this program](https://github.com/apache/spark/pull/16391/files) with 4.7x. Here is a Dataset part of this program.

Without this PR
```
OpenJDK 64-Bit Server VM 1.8.0_111-8u111-b14-2ubuntu0.16.04.2-b14 on Linux 4.4.0-47-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
back-to-back map:                        Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
RDD                                           1923 / 1952         52.0          19.2       1.0X
DataFrame                                      526 /  548        190.2           5.3       3.7X
Dataset                                       3094 / 3154         32.3          30.9       0.6X
```

With this PR
```
OpenJDK 64-Bit Server VM 1.8.0_111-8u111-b14-2ubuntu0.16.04.2-b14 on Linux 4.4.0-47-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
back-to-back map:                        Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
RDD                                           1883 / 1892         53.1          18.8       1.0X
DataFrame                                      502 /  642        199.1           5.0       3.7X
Dataset                                        657 /  784        152.2           6.6       2.9X
```

```java
  def backToBackMap(spark: SparkSession, numRows: Long, numChains: Int): Benchmark = {
    import spark.implicits._
    val rdd = spark.sparkContext.range(0, numRows)
    val ds = spark.range(0, numRows)
    val func = (l: Long) => l + 1
    val benchmark = new Benchmark("back-to-back map", numRows)
...
    benchmark.addCase("Dataset") { iter =>
      var res = ds.as[Long]
      var i = 0
      while (i < numChains) {
        res = res.map(func)
        i += 1
      }
      res.queryExecution.toRdd.foreach(_ => Unit)
    }
    benchmark
  }
```

A motivating example
```java
Seq(1, 2, 3).toDS.map(i => i * 7).show
```

Generated code without this PR
```java
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private scala.collection.Iterator inputadapter_input;
/* 009 */   private UnsafeRow deserializetoobject_result;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder deserializetoobject_holder;
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter deserializetoobject_rowWriter;
/* 012 */   private int mapelements_argValue;
/* 013 */   private UnsafeRow mapelements_result;
/* 014 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder mapelements_holder;
/* 015 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter mapelements_rowWriter;
/* 016 */   private UnsafeRow serializefromobject_result;
/* 017 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 018 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 019 */
/* 020 */   public GeneratedIterator(Object[] references) {
/* 021 */     this.references = references;
/* 022 */   }
/* 023 */
/* 024 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 025 */     partitionIndex = index;
/* 026 */     this.inputs = inputs;
/* 027 */     inputadapter_input = inputs[0];
/* 028 */     deserializetoobject_result = new UnsafeRow(1);
/* 029 */     this.deserializetoobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(deserializetoobject_result, 0);
/* 030 */     this.deserializetoobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(deserializetoobject_holder, 1);
/* 031 */
/* 032 */     mapelements_result = new UnsafeRow(1);
/* 033 */     this.mapelements_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(mapelements_result, 0);
/* 034 */     this.mapelements_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(mapelements_holder, 1);
/* 035 */     serializefromobject_result = new UnsafeRow(1);
/* 036 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 0);
/* 037 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 038 */
/* 039 */   }
/* 040 */
/* 041 */   protected void processNext() throws java.io.IOException {
/* 042 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 043 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 044 */       int inputadapter_value = inputadapter_row.getInt(0);
/* 045 */
/* 046 */       boolean mapelements_isNull = true;
/* 047 */       int mapelements_value = -1;
/* 048 */       if (!false) {
/* 049 */         mapelements_argValue = inputadapter_value;
/* 050 */
/* 051 */         mapelements_isNull = false;
/* 052 */         if (!mapelements_isNull) {
/* 053 */           Object mapelements_funcResult = null;
/* 054 */           mapelements_funcResult = ((scala.Function1) references[0]).apply(mapelements_argValue);
/* 055 */           if (mapelements_funcResult == null) {
/* 056 */             mapelements_isNull = true;
/* 057 */           } else {
/* 058 */             mapelements_value = (Integer) mapelements_funcResult;
/* 059 */           }
/* 060 */
/* 061 */         }
/* 062 */
/* 063 */       }
/* 064 */
/* 065 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 066 */
/* 067 */       if (mapelements_isNull) {
/* 068 */         serializefromobject_rowWriter.setNullAt(0);
/* 069 */       } else {
/* 070 */         serializefromobject_rowWriter.write(0, mapelements_value);
/* 071 */       }
/* 072 */       append(serializefromobject_result);
/* 073 */       if (shouldStop()) return;
/* 074 */     }
/* 075 */   }
/* 076 */ }
```

Generated code with this PR (lines 48-56 are changed)
```java
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private scala.collection.Iterator inputadapter_input;
/* 009 */   private UnsafeRow deserializetoobject_result;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder deserializetoobject_holder;
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter deserializetoobject_rowWriter;
/* 012 */   private int mapelements_argValue;
/* 013 */   private UnsafeRow mapelements_result;
/* 014 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder mapelements_holder;
/* 015 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter mapelements_rowWriter;
/* 016 */   private UnsafeRow serializefromobject_result;
/* 017 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 018 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 019 */
/* 020 */   public GeneratedIterator(Object[] references) {
/* 021 */     this.references = references;
/* 022 */   }
/* 023 */
/* 024 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 025 */     partitionIndex = index;
/* 026 */     this.inputs = inputs;
/* 027 */     inputadapter_input = inputs[0];
/* 028 */     deserializetoobject_result = new UnsafeRow(1);
/* 029 */     this.deserializetoobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(deserializetoobject_result, 0);
/* 030 */     this.deserializetoobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(deserializetoobject_holder, 1);
/* 031 */
/* 032 */     mapelements_result = new UnsafeRow(1);
/* 033 */     this.mapelements_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(mapelements_result, 0);
/* 034 */     this.mapelements_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(mapelements_holder, 1);
/* 035 */     serializefromobject_result = new UnsafeRow(1);
/* 036 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 0);
/* 037 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 038 */
/* 039 */   }
/* 040 */
/* 041 */   protected void processNext() throws java.io.IOException {
/* 042 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 043 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 044 */       int inputadapter_value = inputadapter_row.getInt(0);
/* 045 */
/* 046 */       boolean mapelements_isNull = true;
/* 047 */       int mapelements_value = -1;
/* 048 */       if (!false) {
/* 049 */         mapelements_argValue = inputadapter_value;
/* 050 */
/* 051 */         mapelements_isNull = false;
/* 052 */         if (!mapelements_isNull) {
/* 053 */           mapelements_value = ((scala.Function1) references[0]).apply$mcII$sp(mapelements_argValue);
/* 054 */         }
/* 055 */
/* 056 */       }
/* 057 */
/* 058 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 059 */
/* 060 */       if (mapelements_isNull) {
/* 061 */         serializefromobject_rowWriter.setNullAt(0);
/* 062 */       } else {
/* 063 */         serializefromobject_rowWriter.write(0, mapelements_value);
/* 064 */       }
/* 065 */       append(serializefromobject_result);
/* 066 */       if (shouldStop()) return;
/* 067 */     }
/* 068 */   }
/* 069 */ }
```

Java bytecode for methods for `i => i * 7`
```java
$ javap -c Test\$\$anonfun\$5\$\$anonfun\$apply\$mcV\$sp\$1.class
Compiled from "Test.scala"
public final class org.apache.spark.sql.Test$$anonfun$5$$anonfun$apply$mcV$sp$1 extends scala.runtime.AbstractFunction1$mcII$sp implements scala.Serializable {
  public static final long serialVersionUID;

  public final int apply(int);
    Code:
       0: aload_0
       1: iload_1
       2: invokevirtual #18                 // Method apply$mcII$sp:(I)I
       5: ireturn

  public int apply$mcII$sp(int);
    Code:
       0: iload_1
       1: bipush        7
       3: imul
       4: ireturn

  public final java.lang.Object apply(java.lang.Object);
    Code:
       0: aload_0
       1: aload_1
       2: invokestatic  #29                 // Method scala/runtime/BoxesRunTime.unboxToInt:(Ljava/lang/Object;)I
       5: invokevirtual #31                 // Method apply:(I)I
       8: invokestatic  #35                 // Method scala/runtime/BoxesRunTime.boxToInteger:(I)Ljava/lang/Integer;
      11: areturn

  public org.apache.spark.sql.Test$$anonfun$5$$anonfun$apply$mcV$sp$1(org.apache.spark.sql.Test$$anonfun$5);
    Code:
       0: aload_0
       1: invokespecial #42                 // Method scala/runtime/AbstractFunction1$mcII$sp."<init>":()V
       4: return
}
```
## How was this patch tested?

Added new test suites to `DatasetPrimitiveSuite`.

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #17172 from kiszk/SPARK-19008.
2017-03-09 22:58:52 -08:00
Budde f79371ad86 [SPARK-19611][SQL] Introduce configurable table schema inference
## Summary of changes

Add a new configuration option that allows Spark SQL to infer a case-sensitive schema from a Hive Metastore table's data files when a case-sensitive schema can't be read from the table properties.

- Add spark.sql.hive.caseSensitiveInferenceMode param to SQLConf
- Add schemaPreservesCase field to CatalogTable (set to false when schema can't
  successfully be read from Hive table props)
- Perform schema inference in HiveMetastoreCatalog if schemaPreservesCase is
  false, depending on spark.sql.hive.caseSensitiveInferenceMode
- Add alterTableSchema() method to the ExternalCatalog interface
- Add HiveSchemaInferenceSuite tests
- Refactor and move ParquetFileForamt.meregeMetastoreParquetSchema() as
  HiveMetastoreCatalog.mergeWithMetastoreSchema
- Move schema merging tests from ParquetSchemaSuite to HiveSchemaInferenceSuite

[JIRA for this change](https://issues.apache.org/jira/browse/SPARK-19611)

## How was this patch tested?

The tests in ```HiveSchemaInferenceSuite``` should verify that schema inference is working as expected. ```ExternalCatalogSuite``` has also been extended to cover the new ```alterTableSchema()``` API.

Author: Budde <budde@amazon.com>

Closes #16944 from budde/SPARK-19611.
2017-03-09 12:55:33 -08:00
windpiger 274973d2a3 [SPARK-19763][SQL] qualified external datasource table location stored in catalog
## What changes were proposed in this pull request?

If we create a external datasource table with a non-qualified location , we should qualified it to store in catalog.

```
CREATE TABLE t(a string)
USING parquet
LOCATION '/path/xx'

CREATE TABLE t1(a string, b string)
USING parquet
PARTITIONED BY(b)
LOCATION '/path/xx'
```

when we get the table from catalog, the location should be qualified, e.g.'file:/path/xxx'
## How was this patch tested?
unit test added

Author: windpiger <songjun@outlook.com>

Closes #17095 from windpiger/tablepathQualified.
2017-03-09 01:18:17 -08:00
uncleGen eeb1d6db87 [SPARK-19859][SS][FOLLOW-UP] The new watermark should override the old one.
## What changes were proposed in this pull request?

A follow up to SPARK-19859:

- extract the calculation of `delayMs` and reuse it.
- update EventTimeWatermarkExec
- use the correct `delayMs` in EventTimeWatermark

## How was this patch tested?

Jenkins.

Author: uncleGen <hustyugm@gmail.com>

Closes #17221 from uncleGen/SPARK-19859.
2017-03-08 23:23:10 -08:00
Kunal Khamar 6570cfd7ab [SPARK-19540][SQL] Add ability to clone SparkSession wherein cloned session has an identical copy of the SessionState
Forking a newSession() from SparkSession currently makes a new SparkSession that does not retain SessionState (i.e. temporary tables, SQL config, registered functions etc.) This change adds a method cloneSession() which creates a new SparkSession with a copy of the parent's SessionState.

Subsequent changes to base session are not propagated to cloned session, clone is independent after creation.
If the base is changed after clone has been created, say user registers new UDF, then the new UDF will not be available inside the clone. Same goes for configs and temp tables.

Unit tests

Author: Kunal Khamar <kkhamar@outlook.com>
Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16826 from kunalkhamar/fork-sparksession.
2017-03-08 13:20:45 -08:00
Shixiong Zhu 1bf9012380 [SPARK-19858][SS] Add output mode to flatMapGroupsWithState and disallow invalid cases
## What changes were proposed in this pull request?

Add a output mode parameter to `flatMapGroupsWithState` and just define `mapGroupsWithState` as `flatMapGroupsWithState(Update)`.

`UnsupportedOperationChecker` is modified to disallow unsupported cases.

- Batch mapGroupsWithState or flatMapGroupsWithState is always allowed.
- For streaming (map/flatMap)GroupsWithState, see the following table:

| Operators  | Supported Query Output Mode |
| ------------- | ------------- |
| flatMapGroupsWithState(Update) without aggregation  | Update |
| flatMapGroupsWithState(Update) with aggregation  | None |
| flatMapGroupsWithState(Append) without aggregation  | Append |
| flatMapGroupsWithState(Append) before aggregation  | Append, Update, Complete |
| flatMapGroupsWithState(Append) after aggregation  | None |
| Multiple flatMapGroupsWithState(Append)s  | Append |
| Multiple mapGroupsWithStates  | None |
| Mxing mapGroupsWithStates  and flatMapGroupsWithStates | None |
| Other cases of multiple flatMapGroupsWithState | None |

## How was this patch tested?

The added unit tests. Here are the tests related to (map/flatMap)GroupsWithState:
```
[info] - batch plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on batch relation: supported (1 millisecond)
[info] - batch plan - flatMapGroupsWithState - multiple flatMapGroupsWithState(Append)s on batch relation: supported (0 milliseconds)
[info] - batch plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on batch relation: supported (0 milliseconds)
[info] - batch plan - flatMapGroupsWithState - multiple flatMapGroupsWithState(Update)s on batch relation: supported (0 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation without aggregation in update mode: supported (2 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation without aggregation in append mode: not supported (7 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation without aggregation in complete mode: not supported (5 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation with aggregation in Append mode: not supported (11 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation with aggregation in Update mode: not supported (5 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation with aggregation in Complete mode: not supported (5 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation without aggregation in append mode: supported (1 millisecond)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation without aggregation in update mode: not supported (6 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation before aggregation in Append mode: supported (1 millisecond)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation before aggregation in Update mode: supported (0 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation before aggregation in Complete mode: supported (1 millisecond)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation after aggregation in Append mode: not supported (6 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation after aggregation in Update mode: not supported (4 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation in complete mode: not supported (2 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on batch relation inside streaming relation in Append output mode: supported (1 millisecond)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on batch relation inside streaming relation in Update output mode: supported (1 millisecond)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on batch relation inside streaming relation in Append output mode: supported (0 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on batch relation inside streaming relation in Update output mode: supported (0 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - multiple flatMapGroupsWithStates on streaming relation and all are in append mode: supported (2 milliseconds)
[info] - streaming plan - flatMapGroupsWithState -  multiple flatMapGroupsWithStates on s streaming relation but some are not in append mode: not supported (7 milliseconds)
[info] - streaming plan - mapGroupsWithState - mapGroupsWithState on streaming relation without aggregation in append mode: not supported (3 milliseconds)
[info] - streaming plan - mapGroupsWithState - mapGroupsWithState on streaming relation without aggregation in complete mode: not supported (3 milliseconds)
[info] - streaming plan - mapGroupsWithState - mapGroupsWithState on streaming relation with aggregation in Append mode: not supported (6 milliseconds)
[info] - streaming plan - mapGroupsWithState - mapGroupsWithState on streaming relation with aggregation in Update mode: not supported (3 milliseconds)
[info] - streaming plan - mapGroupsWithState - mapGroupsWithState on streaming relation with aggregation in Complete mode: not supported (4 milliseconds)
[info] - streaming plan - mapGroupsWithState - multiple mapGroupsWithStates on streaming relation and all are in append mode: not supported (4 milliseconds)
[info] - streaming plan - mapGroupsWithState - mixing mapGroupsWithStates and flatMapGroupsWithStates on streaming relation: not supported (4 milliseconds)
```

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #17197 from zsxwing/mapgroups-check.
2017-03-08 13:18:07 -08:00
Wojtek Szymanski e9e2c612d5 [SPARK-19727][SQL] Fix for round function that modifies original column
## What changes were proposed in this pull request?

Fix for SQL round function that modifies original column when underlying data frame is created from a local product.

    import org.apache.spark.sql.functions._

    case class NumericRow(value: BigDecimal)

    val df = spark.createDataFrame(Seq(NumericRow(BigDecimal("1.23456789"))))

    df.show()
    +--------------------+
    |               value|
    +--------------------+
    |1.234567890000000000|
    +--------------------+

    df.withColumn("value_rounded", round('value)).show()

    // before
    +--------------------+-------------+
    |               value|value_rounded|
    +--------------------+-------------+
    |1.000000000000000000|            1|
    +--------------------+-------------+

    // after
    +--------------------+-------------+
    |               value|value_rounded|
    +--------------------+-------------+
    |1.234567890000000000|            1|
    +--------------------+-------------+

## How was this patch tested?

New unit test added to existing suite `org.apache.spark.sql.MathFunctionsSuite`

Author: Wojtek Szymanski <wk.szymanski@gmail.com>

Closes #17075 from wojtek-szymanski/SPARK-19727.
2017-03-08 12:36:16 -08:00
Xiao Li 9a6ac7226f [SPARK-19601][SQL] Fix CollapseRepartition rule to preserve shuffle-enabled Repartition
### What changes were proposed in this pull request?

Observed by felixcheung  in https://github.com/apache/spark/pull/16739, when users use the shuffle-enabled `repartition` API, they expect the partition they got should be the exact number they provided, even if they call shuffle-disabled `coalesce` later.

Currently, `CollapseRepartition` rule does not consider whether shuffle is enabled or not. Thus, we got the following unexpected result.

```Scala
    val df = spark.range(0, 10000, 1, 5)
    val df2 = df.repartition(10)
    assert(df2.coalesce(13).rdd.getNumPartitions == 5)
    assert(df2.coalesce(7).rdd.getNumPartitions == 5)
    assert(df2.coalesce(3).rdd.getNumPartitions == 3)
```

This PR is to fix the issue. We preserve shuffle-enabled Repartition.

### How was this patch tested?
Added a test case

Author: Xiao Li <gatorsmile@gmail.com>

Closes #16933 from gatorsmile/CollapseRepartition.
2017-03-08 09:36:01 -08:00
jiangxingbo 5f7d835d38 [SPARK-19865][SQL] remove the view identifier in SubqueryAlias
## What changes were proposed in this pull request?

Since we have a `View` node now, we can remove the view identifier in `SubqueryAlias`, which was used to indicate a view node before.

## How was this patch tested?

Update the related test cases.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #17210 from jiangxb1987/SubqueryAlias.
2017-03-08 16:18:17 +01:00
wangzhenhua e44274870d [SPARK-17080][SQL] join reorder
## What changes were proposed in this pull request?

Reorder the joins using a dynamic programming algorithm (Selinger paper):
First we put all items (basic joined nodes) into level 1, then we build all two-way joins at level 2 from plans at level 1 (single items), then build all 3-way joins from plans at previous levels (two-way joins and single items), then 4-way joins ... etc, until we build all n-way joins and pick the best plan among them.

When building m-way joins, we only keep the best plan (with the lowest cost) for the same set of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among plans (A J B) J C, (A J C) J B and (B J C) J A. Thus, the plans maintained for each level when reordering four items A, B, C, D are as follows:
```
level 1: p({A}), p({B}), p({C}), p({D})
level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
level 4: p({A, B, C, D})
```
where p({A, B, C, D}) is the final output plan.

For cost evaluation, since physical costs for operators are not available currently, we use cardinalities and sizes to compute costs.

## How was this patch tested?
add test cases

Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>

Closes #17138 from wzhfy/joinReorder.
2017-03-08 16:01:28 +01:00
Michael Armbrust 314e48a358 [SPARK-18055][SQL] Use correct mirror in ExpresionEncoder
Previously, we were using the mirror of passed in `TypeTag` when reflecting to build an encoder.  This fails when the outer class is built in (i.e. `Seq`'s default mirror is based on root classloader) but inner classes (i.e. `A` in `Seq[A]`) are defined in the REPL or a library.

This patch changes us to always reflect based on a mirror created using the context classloader.

Author: Michael Armbrust <michael@databricks.com>

Closes #17201 from marmbrus/replSeqEncoder.
2017-03-08 01:32:42 -08:00
Shixiong Zhu d8830c5039 [SPARK-19859][SS] The new watermark should override the old one
## What changes were proposed in this pull request?

The new watermark should override the old one. Otherwise, we just pick up the first column which has a watermark, it may be unexpected.

## How was this patch tested?

The new test.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #17199 from zsxwing/SPARK-19859.
2017-03-07 20:34:55 -08:00
Tejas Patil c96d14abae [SPARK-19843][SQL] UTF8String => (int / long) conversion expensive for invalid inputs
## What changes were proposed in this pull request?

Jira : https://issues.apache.org/jira/browse/SPARK-19843

Created wrapper classes (`IntWrapper`, `LongWrapper`) to wrap the result of parsing (which are primitive types). In case of problem in parsing, the method would return a boolean.

## How was this patch tested?

- Added new unit tests
- Ran a prod job which had conversion from string -> int and verified the outputs

## Performance

Tiny regression when all strings are valid integers

```
conversion to int:       Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
--------------------------------------------------------------------------------
trunk                         502 /  522         33.4          29.9       1.0X
SPARK-19843                   493 /  503         34.0          29.4       1.0X
```

Huge gain when all strings are invalid integers
```
conversion to int:      Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
-------------------------------------------------------------------------------
trunk                     33913 / 34219          0.5        2021.4       1.0X
SPARK-19843                  154 /  162        108.8           9.2     220.0X
```

Author: Tejas Patil <tejasp@fb.com>

Closes #17184 from tejasapatil/SPARK-19843_is_numeric_maybe.
2017-03-07 20:19:30 -08:00
Takeshi Yamamuro 030acdd1f0 [SPARK-19637][SQL] Add to_json in FunctionRegistry
## What changes were proposed in this pull request?
This pr added entries  in `FunctionRegistry` and supported `to_json` in SQL.

## How was this patch tested?
Added tests in `JsonFunctionsSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #16981 from maropu/SPARK-19637.
2017-03-07 09:00:14 -08:00
wangzhenhua 932196d9e3 [SPARK-17075][SQL][FOLLOWUP] fix filter estimation issues
## What changes were proposed in this pull request?

1. support boolean type in binary expression estimation.
2. deal with compound Not conditions.
3. avoid convert BigInt/BigDecimal directly to double unless it's within range (0, 1).
4. reorganize test code.

## How was this patch tested?

modify related test cases.

Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>

Closes #17148 from wzhfy/fixFilter.
2017-03-06 23:53:53 -08:00
wangzhenhua 9909f6d361 [SPARK-19350][SQL] Cardinality estimation of Limit and Sample
## What changes were proposed in this pull request?

Before this pr, LocalLimit/GlobalLimit/Sample propagates the same row count and column stats from its child, which is incorrect.
We can get the correct rowCount in Statistics for GlobalLimit/Sample whether cbo is enabled or not.
We don't know the rowCount for LocalLimit because we don't know the partition number at that time. Column stats should not be propagated because we don't know the distribution of columns after Limit or Sample.

## How was this patch tested?

Added test cases.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #16696 from wzhfy/limitEstimation.
2017-03-06 21:45:36 -08:00
jiangxingbo 9991c2dad6 [SPARK-19211][SQL] Explicitly prevent Insert into View or Create View As Insert
## What changes were proposed in this pull request?

Currently we don't explicitly forbid the following behaviors:
1. The statement CREATE VIEW AS INSERT INTO throws the following exception:
```
scala> spark.sql("CREATE VIEW testView AS INSERT INTO tab VALUES (1, \"a\")")
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: org.apache.hadoop.hive.ql.metadata.HiveException: at least one column must be specified for the table;
 scala> spark.sql("CREATE VIEW testView(a, b) AS INSERT INTO tab VALUES (1, \"a\")")
org.apache.spark.sql.AnalysisException: The number of columns produced by the SELECT clause (num: `0`) does not match the number of column names specified by CREATE VIEW (num: `2`).;
```

2. The statement INSERT INTO view VALUES throws the following exception from checkAnalysis:
```
scala> spark.sql("INSERT INTO testView VALUES (1, \"a\")")
org.apache.spark.sql.AnalysisException: Inserting into an RDD-based table is not allowed.;;
'InsertIntoTable View (`default`.`testView`, [a#16,b#17]), false, false
+- LocalRelation [col1#14, col2#15]
```

After this PR, the behavior changes to:
```
scala> spark.sql("CREATE VIEW testView AS INSERT INTO tab VALUES (1, \"a\")")
org.apache.spark.sql.catalyst.parser.ParseException: Operation not allowed: CREATE VIEW ... AS INSERT INTO;

scala> spark.sql("CREATE VIEW testView(a, b) AS INSERT INTO tab VALUES (1, \"a\")")
org.apache.spark.sql.catalyst.parser.ParseException: Operation not allowed: CREATE VIEW ... AS INSERT INTO;

scala> spark.sql("INSERT INTO testView VALUES (1, \"a\")")
org.apache.spark.sql.AnalysisException: `default`.`testView` is a view, inserting into a view is not allowed;
```

## How was this patch tested?

Add a new test case in `SparkSqlParserSuite`;
Update the corresponding test case in `SQLViewSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #17125 from jiangxb1987/insert-with-view.
2017-03-06 12:35:03 -08:00
windpiger 096df6d933 [SPARK-19257][SQL] location for table/partition/database should be java.net.URI
## What changes were proposed in this pull request?

Currently we treat the location of table/partition/database as URI string.

It will be safer if we can make the type of location as java.net.URI.

In this PR, there are following classes changes:
**1. CatalogDatabase**
```
case class CatalogDatabase(
    name: String,
    description: String,
    locationUri: String,
    properties: Map[String, String])
--->
case class CatalogDatabase(
    name: String,
    description: String,
    locationUri: URI,
    properties: Map[String, String])
```
**2. CatalogStorageFormat**
```
case class CatalogStorageFormat(
    locationUri: Option[String],
    inputFormat: Option[String],
    outputFormat: Option[String],
    serde: Option[String],
    compressed: Boolean,
    properties: Map[String, String])
---->
case class CatalogStorageFormat(
    locationUri: Option[URI],
    inputFormat: Option[String],
    outputFormat: Option[String],
    serde: Option[String],
    compressed: Boolean,
    properties: Map[String, String])
```

Before and After this PR, it is transparent for user, there is no change that the user should concern. The `String` to `URI` just happened in SparkSQL internally.

Here list some operation related location:
**1. whitespace in the location**
   e.g.  `/a/b c/d`
   For both table location and partition location,
   After `CREATE TABLE  t... (PARTITIONED BY ...) LOCATION '/a/b c/d'` ,
   then `DESC EXTENDED t ` show the location is `/a/b c/d`,
   and the real path in the FileSystem also show `/a/b c/d`

**2. colon(:) in the location**
   e.g.  `/a/b:c/d`
   For both table location and partition location,
   when `CREATE TABLE  t... (PARTITIONED BY ...)  LOCATION '/a/b:c/d'` ,

  **In linux file system**
   `DESC EXTENDED t ` show the location is `/a/b:c/d`,
   and the real path in the FileSystem also show `/a/b:c/d`

  **in HDFS** throw exception:
  `java.lang.IllegalArgumentException: Pathname /a/b:c/d from hdfs://iZbp1151s8hbnnwriekxdeZ:9000/a/b:c/d is not a valid DFS filename.`

  **while** After `INSERT INTO TABLE t PARTITION(a="a:b") SELECT 1`
   then `DESC EXTENDED t ` show the location is `/xxx/a=a%3Ab`,
   and the real path in the FileSystem also show `/xxx/a=a%3Ab`

**3. percent sign(%) in the location**
   e.g.  `/a/b%c/d`
   For both table location and partition location,
   After `CREATE TABLE  t... (PARTITIONED BY ...) LOCATION '/a/b%c/d'` ,
   then `DESC EXTENDED t ` show the location is `/a/b%c/d`,
   and the real path in the FileSystem also show `/a/b%c/d`

**4. encoded(%25) in the location**
   e.g.  `/a/b%25c/d`
   For both table location and partition location,
   After `CREATE TABLE  t... (PARTITIONED BY ...)  LOCATION '/a/b%25c/d'` ,
   then `DESC EXTENDED t ` show the location is `/a/b%25c/d`,
   and the real path in the FileSystem also show `/a/b%25c/d`

   **while** After `INSERT INTO TABLE t PARTITION(a="%25") SELECT 1`
   then `DESC EXTENDED t ` show the location is `/xxx/a=%2525`,
   and the real path in the FileSystem also show `/xxx/a=%2525`

**Additionally**, except the location, there are two other factors will affect the location of the table/partition. one is the table name which does not allowed to have special characters, and the  other is `partition name` which have the same actions with `partition value`, and `partition name` with special character situation has add some testcase and resolve a bug in [PR](https://github.com/apache/spark/pull/17173)

### Summary:
After `CREATE TABLE  t... (PARTITIONED BY ...)  LOCATION path`,
the path which we get from `DESC TABLE` and `real path in FileSystem` are all the same with the `CREATE TABLE` command(different filesystem has different action that allow what kind of special character to create the path, e.g. HDFS does not allow colon, but linux filesystem allow it ).

`DataBase` also have the same logic with `CREATE TABLE`

while if the `partition value` has some special character like `%` `:` `#` etc, then we will get the path with encoded `partition value` like `/xxx/a=A%25B` from `DESC TABLE` and `real path in FileSystem`

In this PR, the core change code is using `new Path(str).toUri` and `new Path(uri).toString`
which transfrom `str to uri `or `uri to str`.
for example:
```
val str = '/a/b c/d'
val uri = new Path(str).toUri  --> '/a/b%20c/d'
val strFromUri = new Path(uri).toString -> '/a/b c/d'
```

when we restore table/partition from metastore, or get the location from `CREATE TABLE` command, we can use it as above to change string to uri `new Path(str).toUri `

## How was this patch tested?
unit test added.
The `current master branch` also `passed all the test cases` added in this PR by a litter change.
https://github.com/apache/spark/pull/17149/files#diff-b7094baa12601424a5d19cb930e3402fR1764
here `toURI` -> `toString` when test in master branch.

This can show that this PR  is transparent for user.

Author: windpiger <songjun@outlook.com>

Closes #17149 from windpiger/changeStringToURI.
2017-03-06 10:44:26 -08:00
Cheng Lian 339b53a131 [SPARK-19737][SQL] New analysis rule for reporting unregistered functions without relying on relation resolution
## What changes were proposed in this pull request?

This PR adds a new `Once` analysis rule batch consists of a single analysis rule `LookupFunctions` that performs simple existence check over `UnresolvedFunctions` without actually resolving them.

The benefit of this rule is that it doesn't require function arguments to be resolved first and therefore doesn't rely on relation resolution, which may incur potentially expensive partition/schema discovery cost.

Please refer to [SPARK-19737][1] for more details about the motivation.

## How was this patch tested?

New test case added in `AnalysisErrorSuite`.

[1]: https://issues.apache.org/jira/browse/SPARK-19737

Author: Cheng Lian <lian@databricks.com>

Closes #17168 from liancheng/spark-19737-lookup-functions.
2017-03-06 10:36:50 -08:00
Tejas Patil 2a0bc867a4 [SPARK-17495][SQL] Support Decimal type in Hive-hash
## What changes were proposed in this pull request?

Hive hash to support Decimal datatype. [Hive internally normalises decimals](4ba713ccd8/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimalV1.java (L307)) and I have ported that logic as-is to HiveHash.

## How was this patch tested?

Added unit tests

Author: Tejas Patil <tejasp@fb.com>

Closes #17056 from tejasapatil/SPARK-17495_decimal.
2017-03-06 10:16:20 -08:00
hyukjinkwon 369a148e59 [SPARK-19595][SQL] Support json array in from_json
## What changes were proposed in this pull request?

This PR proposes to both,

**Do not allow json arrays with multiple elements and return null in `from_json` with `StructType` as the schema.**

Currently, it only reads the single row when the input is a json array. So, the codes below:

```scala
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._
val schema = StructType(StructField("a", IntegerType) :: Nil)
Seq(("""[{"a": 1}, {"a": 2}]""")).toDF("struct").select(from_json(col("struct"), schema)).show()
```
prints

```
+--------------------+
|jsontostruct(struct)|
+--------------------+
|                 [1]|
+--------------------+
```

This PR simply suggests to print this as `null` if the schema is `StructType` and input is json array.with multiple elements

```
+--------------------+
|jsontostruct(struct)|
+--------------------+
|                null|
+--------------------+
```

**Support json arrays in `from_json` with `ArrayType` as the schema.**

```scala
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._
val schema = ArrayType(StructType(StructField("a", IntegerType) :: Nil))
Seq(("""[{"a": 1}, {"a": 2}]""")).toDF("array").select(from_json(col("array"), schema)).show()
```

prints

```
+-------------------+
|jsontostruct(array)|
+-------------------+
|         [[1], [2]]|
+-------------------+
```

## How was this patch tested?

Unit test in `JsonExpressionsSuite`, `JsonFunctionsSuite`, Python doctests and manual test.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16929 from HyukjinKwon/disallow-array.
2017-03-05 14:35:06 -08:00
Takeshi Yamamuro 14bb398fae [SPARK-19254][SQL] Support Seq, Map, and Struct in functions.lit
## What changes were proposed in this pull request?
This pr is to support Seq, Map, and Struct in functions.lit; it adds a new IF named `lit2` with `TypeTag` for avoiding type erasure.

## How was this patch tested?
Added tests in `LiteralExpressionSuite`

Author: Takeshi Yamamuro <yamamuro@apache.org>
Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes #16610 from maropu/SPARK-19254.
2017-03-05 03:53:19 -08:00
Takuya UESHIN 2a7921a813 [SPARK-18939][SQL] Timezone support in partition values.
## What changes were proposed in this pull request?

This is a follow-up pr of #16308 and #16750.

This pr enables timezone support in partition values.

We should use `timeZone` option introduced at #16750 to parse/format partition values of the `TimestampType`.

For example, if you have timestamp `"2016-01-01 00:00:00"` in `GMT` which will be used for partition values, the values written by the default timezone option, which is `"GMT"` because the session local timezone is `"GMT"` here, are:

```scala
scala> spark.conf.set("spark.sql.session.timeZone", "GMT")

scala> val df = Seq((1, new java.sql.Timestamp(1451606400000L))).toDF("i", "ts")
df: org.apache.spark.sql.DataFrame = [i: int, ts: timestamp]

scala> df.show()
+---+-------------------+
|  i|                 ts|
+---+-------------------+
|  1|2016-01-01 00:00:00|
+---+-------------------+

scala> df.write.partitionBy("ts").save("/path/to/gmtpartition")
```

```sh
$ ls /path/to/gmtpartition/
_SUCCESS			ts=2016-01-01 00%3A00%3A00
```

whereas setting the option to `"PST"`, they are:

```scala
scala> df.write.option("timeZone", "PST").partitionBy("ts").save("/path/to/pstpartition")
```

```sh
$ ls /path/to/pstpartition/
_SUCCESS			ts=2015-12-31 16%3A00%3A00
```

We can properly read the partition values if the session local timezone and the timezone of the partition values are the same:

```scala
scala> spark.read.load("/path/to/gmtpartition").show()
+---+-------------------+
|  i|                 ts|
+---+-------------------+
|  1|2016-01-01 00:00:00|
+---+-------------------+
```

And even if the timezones are different, we can properly read the values with setting corrent timezone option:

```scala
// wrong result
scala> spark.read.load("/path/to/pstpartition").show()
+---+-------------------+
|  i|                 ts|
+---+-------------------+
|  1|2015-12-31 16:00:00|
+---+-------------------+

// correct result
scala> spark.read.option("timeZone", "PST").load("/path/to/pstpartition").show()
+---+-------------------+
|  i|                 ts|
+---+-------------------+
|  1|2016-01-01 00:00:00|
+---+-------------------+
```

## How was this patch tested?

Existing tests and added some tests.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #17053 from ueshin/issues/SPARK-18939.
2017-03-03 16:35:54 -08:00
Liang-Chi Hsieh 98bcc188f9 [SPARK-19758][SQL] Resolving timezone aware expressions with time zone when resolving inline table
## What changes were proposed in this pull request?

When we resolve inline tables in analyzer, we will evaluate the expressions of inline tables.

When it evaluates a `TimeZoneAwareExpression` expression, an error will happen because the `TimeZoneAwareExpression` is not associated with timezone yet.

So we need to resolve these `TimeZoneAwareExpression`s with time zone when resolving inline tables.

## How was this patch tested?

Jenkins tests.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #17114 from viirya/resolve-timeawareexpr-inline-table.
2017-03-03 07:14:37 -08:00
Stan Zhai 5502a9cf88 [SPARK-19766][SQL] Constant alias columns in INNER JOIN should not be folded by FoldablePropagation rule
## What changes were proposed in this pull request?
This PR fixes the code in Optimizer phase where the constant alias columns of a `INNER JOIN` query are folded in Rule `FoldablePropagation`.

For the following query():

```
val sqlA =
  """
    |create temporary view ta as
    |select a, 'a' as tag from t1 union all
    |select a, 'b' as tag from t2
  """.stripMargin

val sqlB =
  """
    |create temporary view tb as
    |select a, 'a' as tag from t3 union all
    |select a, 'b' as tag from t4
  """.stripMargin

val sql =
  """
    |select tb.* from ta inner join tb on
    |ta.a = tb.a and
    |ta.tag = tb.tag
  """.stripMargin
```

The tag column is an constant alias column, it's folded by `FoldablePropagation` like this:

```
TRACE SparkOptimizer:
=== Applying Rule org.apache.spark.sql.catalyst.optimizer.FoldablePropagation ===
 Project [a#4, tag#14]                              Project [a#4, tag#14]
!+- Join Inner, ((a#0 = a#4) && (tag#8 = tag#14))   +- Join Inner, ((a#0 = a#4) && (a = a))
    :- Union                                           :- Union
    :  :- Project [a#0, a AS tag#8]                    :  :- Project [a#0, a AS tag#8]
    :  :  +- LocalRelation [a#0]                       :  :  +- LocalRelation [a#0]
    :  +- Project [a#2, b AS tag#9]                    :  +- Project [a#2, b AS tag#9]
    :     +- LocalRelation [a#2]                       :     +- LocalRelation [a#2]
    +- Union                                           +- Union
       :- Project [a#4, a AS tag#14]                      :- Project [a#4, a AS tag#14]
       :  +- LocalRelation [a#4]                          :  +- LocalRelation [a#4]
       +- Project [a#6, b AS tag#15]                      +- Project [a#6, b AS tag#15]
          +- LocalRelation [a#6]                             +- LocalRelation [a#6]
```

Finally the Result of Batch Operator Optimizations is:

```
Project [a#4, tag#14]                              Project [a#4, tag#14]
!+- Join Inner, ((a#0 = a#4) && (tag#8 = tag#14))   +- Join Inner, (a#0 = a#4)
!   :- SubqueryAlias ta, `ta`                          :- Union
!   :  +- Union                                        :  :- LocalRelation [a#0]
!   :     :- Project [a#0, a AS tag#8]                 :  +- LocalRelation [a#2]
!   :     :  +- SubqueryAlias t1, `t1`                 +- Union
!   :     :     +- Project [a#0]                          :- LocalRelation [a#4, tag#14]
!   :     :        +- SubqueryAlias grouping              +- LocalRelation [a#6, tag#15]
!   :     :           +- LocalRelation [a#0]
!   :     +- Project [a#2, b AS tag#9]
!   :        +- SubqueryAlias t2, `t2`
!   :           +- Project [a#2]
!   :              +- SubqueryAlias grouping
!   :                 +- LocalRelation [a#2]
!   +- SubqueryAlias tb, `tb`
!      +- Union
!         :- Project [a#4, a AS tag#14]
!         :  +- SubqueryAlias t3, `t3`
!         :     +- Project [a#4]
!         :        +- SubqueryAlias grouping
!         :           +- LocalRelation [a#4]
!         +- Project [a#6, b AS tag#15]
!            +- SubqueryAlias t4, `t4`
!               +- Project [a#6]
!                  +- SubqueryAlias grouping
!                     +- LocalRelation [a#6]
```

The condition `tag#8 = tag#14` of INNER JOIN has been removed. This leads to the data of inner join being wrong.

After fix:

```
=== Result of Batch LocalRelation ===
 GlobalLimit 21                                           GlobalLimit 21
 +- LocalLimit 21                                         +- LocalLimit 21
    +- Project [a#4, tag#11]                                 +- Project [a#4, tag#11]
       +- Join Inner, ((a#0 = a#4) && (tag#8 = tag#11))         +- Join Inner, ((a#0 = a#4) && (tag#8 = tag#11))
!         :- SubqueryAlias ta                                      :- Union
!         :  +- Union                                              :  :- LocalRelation [a#0, tag#8]
!         :     :- Project [a#0, a AS tag#8]                       :  +- LocalRelation [a#2, tag#9]
!         :     :  +- SubqueryAlias t1                             +- Union
!         :     :     +- Project [a#0]                                :- LocalRelation [a#4, tag#11]
!         :     :        +- SubqueryAlias grouping                    +- LocalRelation [a#6, tag#12]
!         :     :           +- LocalRelation [a#0]
!         :     +- Project [a#2, b AS tag#9]
!         :        +- SubqueryAlias t2
!         :           +- Project [a#2]
!         :              +- SubqueryAlias grouping
!         :                 +- LocalRelation [a#2]
!         +- SubqueryAlias tb
!            +- Union
!               :- Project [a#4, a AS tag#11]
!               :  +- SubqueryAlias t3
!               :     +- Project [a#4]
!               :        +- SubqueryAlias grouping
!               :           +- LocalRelation [a#4]
!               +- Project [a#6, b AS tag#12]
!                  +- SubqueryAlias t4
!                     +- Project [a#6]
!                        +- SubqueryAlias grouping
!                           +- LocalRelation [a#6]
```

## How was this patch tested?

add sql-tests/inputs/inner-join.sql
All tests passed.

Author: Stan Zhai <zhaishidan@haizhi.com>

Closes #17099 from stanzhai/fix-inner-join.
2017-03-01 07:52:35 -08:00
Wenchen Fan 7c7fc30b4a [SPARK-19678][SQL] remove MetastoreRelation
## What changes were proposed in this pull request?

`MetastoreRelation` is used to represent table relation for hive tables, and provides some hive related information. We will resolve `SimpleCatalogRelation` to `MetastoreRelation` for hive tables, which is unnecessary as these 2 are the same essentially. This PR merges `SimpleCatalogRelation` and `MetastoreRelation`

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17015 from cloud-fan/table-relation.
2017-02-28 09:24:36 -08:00
hyukjinkwon 4ba9c6c453 [MINOR][BUILD] Fix lint-java breaks in Java
## What changes were proposed in this pull request?

This PR proposes to fix the lint-breaks as below:

```
[ERROR] src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java:[29,8] (imports) UnusedImports: Unused import - org.apache.spark.network.buffer.ManagedBuffer.
[ERROR] src/main/java/org/apache/spark/unsafe/types/UTF8String.java:[156,10] (modifier) ModifierOrder: 'Nonnull' annotation modifier does not precede non-annotation modifiers.
[ERROR] src/main/java/org/apache/spark/SparkFirehoseListener.java:[122] (sizes) LineLength: Line is longer than 100 characters (found 105).
[ERROR] src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java:[164,78] (coding) OneStatementPerLine: Only one statement per line allowed.
[ERROR] src/test/java/test/org/apache/spark/JavaAPISuite.java:[1157] (sizes) LineLength: Line is longer than 100 characters (found 121).
[ERROR] src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java:[149] (sizes) LineLength: Line is longer than 100 characters (found 113).
[ERROR] src/test/java/test/org/apache/spark/streaming/Java8APISuite.java:[146] (sizes) LineLength: Line is longer than 100 characters (found 122).
[ERROR] src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java:[32,8] (imports) UnusedImports: Unused import - org.apache.spark.streaming.Time.
[ERROR] src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java:[611] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java:[1317] (sizes) LineLength: Line is longer than 100 characters (found 102).
[ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuite.java:[91] (sizes) LineLength: Line is longer than 100 characters (found 102).
[ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java:[113] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java:[164] (sizes) LineLength: Line is longer than 100 characters (found 110).
[ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java:[212] (sizes) LineLength: Line is longer than 100 characters (found 114).
[ERROR] src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java:[36] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java:[26,8] (imports) UnusedImports: Unused import - com.amazonaws.regions.RegionUtils.
[ERROR] src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java:[20,8] (imports) UnusedImports: Unused import - com.amazonaws.regions.RegionUtils.
[ERROR] src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java:[94] (sizes) LineLength: Line is longer than 100 characters (found 103).
[ERROR] src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java:[30,8] (imports) UnusedImports: Unused import - org.apache.spark.sql.api.java.UDF1.
[ERROR] src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java:[72] (sizes) LineLength: Line is longer than 100 characters (found 104).
[ERROR] src/main/java/org/apache/spark/examples/mllib/JavaRankingMetricsExample.java:[121] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java:[28,8] (imports) UnusedImports: Unused import - org.apache.spark.api.java.JavaRDD.
[ERROR] src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java:[29,8] (imports) UnusedImports: Unused import - org.apache.spark.api.java.JavaSparkContext.
```

## How was this patch tested?

Manually via

```bash
./dev/lint-java
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17072 from HyukjinKwon/java-lint.
2017-02-27 08:44:26 +00:00
Wenchen Fan 89608cf262 [SPARK-17075][SQL][FOLLOWUP] fix some minor issues and clean up the code
## What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/16395. It fixes some code style issues, naming issues, some missing cases in pattern match, etc.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17065 from cloud-fan/follow-up.
2017-02-25 23:01:44 -08:00
Xiao Li 4cb025afaf [SPARK-19735][SQL] Remove HOLD_DDLTIME from Catalog APIs
### What changes were proposed in this pull request?
As explained in Hive JIRA https://issues.apache.org/jira/browse/HIVE-12224, HOLD_DDLTIME was broken as soon as it landed. Hive 2.0 removes HOLD_DDLTIME from the API. In Spark SQL, we always set it to FALSE. Like Hive, we should also remove it from our Catalog APIs.

### How was this patch tested?
N/A

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17063 from gatorsmile/removalHoldDDLTime.
2017-02-24 23:03:59 -08:00
wangzhenhua 69d0da6373 [SPARK-17078][SQL] Show stats when explain
## What changes were proposed in this pull request?

Currently we can only check the estimated stats in logical plans by debugging. We need to provide an easier and more efficient way for developers/users.

In this pr, we add EXPLAIN COST command to show stats in the optimized logical plan.
E.g.
```
spark-sql> EXPLAIN COST select count(1) from store_returns;

...
== Optimized Logical Plan ==
Aggregate [count(1) AS count(1)#24L], Statistics(sizeInBytes=16.0 B, rowCount=1, isBroadcastable=false)
+- Project, Statistics(sizeInBytes=4.3 GB, rowCount=5.76E+8, isBroadcastable=false)
   +- Relation[sr_returned_date_sk#3,sr_return_time_sk#4,sr_item_sk#5,sr_customer_sk#6,sr_cdemo_sk#7,sr_hdemo_sk#8,sr_addr_sk#9,sr_store_sk#10,sr_reason_sk#11,sr_ticket_number#12,sr_return_quantity#13,sr_return_amt#14,sr_return_tax#15,sr_return_amt_inc_tax#16,sr_fee#17,sr_return_ship_cost#18,sr_refunded_cash#19,sr_reversed_charge#20,sr_store_credit#21,sr_net_loss#22] parquet, Statistics(sizeInBytes=28.6 GB, rowCount=5.76E+8, isBroadcastable=false)
...
```

## How was this patch tested?

Add test cases.

Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>

Closes #16594 from wzhfy/showStats.
2017-02-24 10:24:59 -08:00
Shuai Lin 05954f32e9 [SPARK-17075][SQL] Follow up: fix file line ending and improve the tests
## What changes were proposed in this pull request?

Fixed the line ending of `FilterEstimation.scala` (It's still using `\n\r`). Also improved the tests to cover the cases where the literals are on the left side of a binary operator.

## How was this patch tested?

Existing unit tests.

Author: Shuai Lin <linshuai2012@gmail.com>

Closes #17051 from lins05/fix-cbo-filter-file-encoding.
2017-02-24 10:24:01 -08:00
Tejas Patil 3e40f6c3d6 [SPARK-17495][SQL] Add more tests for hive hash
## What changes were proposed in this pull request?

This PR adds tests hive-hash by comparing the outputs generated against Hive 1.2.1. Following datatypes are covered by this PR:
- null
- boolean
- byte
- short
- int
- long
- float
- double
- string
- array
- map
- struct

Datatypes that I have _NOT_ covered but I will work on separately are:
- Decimal (handled separately in https://github.com/apache/spark/pull/17056)
- TimestampType
- DateType
- CalendarIntervalType

## How was this patch tested?

NA

Author: Tejas Patil <tejasp@fb.com>

Closes #17049 from tejasapatil/SPARK-17495_remaining_types.
2017-02-24 09:46:42 -08:00
Ron Hu d7e43b613a [SPARK-17075][SQL] implemented filter estimation
## What changes were proposed in this pull request?

We traverse predicate and evaluate the logical expressions to compute the selectivity of a FILTER operator.

## How was this patch tested?

We add a new test suite to test various logical operators.

Author: Ron Hu <ron.hu@huawei.com>

Closes #16395 from ron8hu/filterSelectivity.
2017-02-23 20:18:21 -08:00
Shixiong Zhu 9bf4e2baad [SPARK-19497][SS] Implement streaming deduplication
## What changes were proposed in this pull request?

This PR adds a special streaming deduplication operator to support `dropDuplicates` with `aggregation` and watermark. It reuses the `dropDuplicates` API but creates new logical plan `Deduplication` and new physical plan `DeduplicationExec`.

The following cases are supported:

- one or multiple `dropDuplicates()` without aggregation (with or without watermark)
- `dropDuplicates` before aggregation

Not supported cases:

- `dropDuplicates` after aggregation

Breaking changes:
- `dropDuplicates` without aggregation doesn't work with `complete` or `update` mode.

## How was this patch tested?

The new unit tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16970 from zsxwing/dedup.
2017-02-23 11:25:39 -08:00
Herman van Hovell 78eae7e67f [SPARK-19459] Support for nested char/varchar fields in ORC
## What changes were proposed in this pull request?
This PR is a small follow-up on https://github.com/apache/spark/pull/16804. This PR also adds support for nested char/varchar fields in orc.

## How was this patch tested?
I have added a regression test to the OrcSourceSuite.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17030 from hvanhovell/SPARK-19459-follow-up.
2017-02-23 10:25:18 -08:00
Takeshi Yamamuro 93aa427159 [SPARK-19691][SQL] Fix ClassCastException when calculating percentile of decimal column
## What changes were proposed in this pull request?
This pr fixed a class-cast exception below;
```
scala> spark.range(10).selectExpr("cast (id as decimal) as x").selectExpr("percentile(x, 0.5)").collect()
 java.lang.ClassCastException: org.apache.spark.sql.types.Decimal cannot be cast to java.lang.Number
	at org.apache.spark.sql.catalyst.expressions.aggregate.Percentile.update(Percentile.scala:141)
	at org.apache.spark.sql.catalyst.expressions.aggregate.Percentile.update(Percentile.scala:58)
	at org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggregate.update(interfaces.scala:514)
	at org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$1$$anonfun$applyOrElse$1.apply(AggregationIterator.scala:171)
	at org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$1$$anonfun$applyOrElse$1.apply(AggregationIterator.scala:171)
	at org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$generateProcessRow$1.apply(AggregationIterator.scala:187)
	at org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$generateProcessRow$1.apply(AggregationIterator.scala:181)
	at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.processInputs(ObjectAggregationIterator.scala:151)
	at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.<init>(ObjectAggregationIterator.scala:78)
	at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec$$anonfun$doExecute$1$$anonfun$2.apply(ObjectHashAggregateExec.scala:109)
	at
```
This fix simply converts catalyst values (i.e., `Decimal`) into scala ones by using `CatalystTypeConverters`.

## How was this patch tested?
Added a test in `DataFrameSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17028 from maropu/SPARK-19691.
2017-02-23 16:28:36 +01:00
Takeshi Yamamuro 769aa0f1d2 [SPARK-19695][SQL] Throw an exception if a columnNameOfCorruptRecord field violates requirements in json formats
## What changes were proposed in this pull request?
This pr comes from #16928 and fixed a json behaviour along with the CSV one.

## How was this patch tested?
Added tests in `JsonSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17023 from maropu/SPARK-19695.
2017-02-22 21:39:20 -08:00
Xiao Li dc005ed53c [SPARK-19658][SQL] Set NumPartitions of RepartitionByExpression In Parser
### What changes were proposed in this pull request?

Currently, if `NumPartitions` is not set in RepartitionByExpression, we will set it using `spark.sql.shuffle.partitions` during Planner. However, this is not following the general resolution process. This PR is to set it in `Parser` and then `Optimizer` can use the value for plan optimization.

### How was this patch tested?

Added a test case.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #16988 from gatorsmile/resolveRepartition.
2017-02-22 17:26:56 -08:00
hyukjinkwon 37112fcfcd [SPARK-19666][SQL] Skip a property without getter in Java schema inference and allow empty bean in encoder creation
## What changes were proposed in this pull request?

This PR proposes to fix two.

**Skip a property without a getter in beans**

Currently, if we use a JavaBean without the getter as below:

```java
public static class BeanWithoutGetter implements Serializable {
  private String a;

  public void setA(String a) {
    this.a = a;
  }
}

BeanWithoutGetter bean = new BeanWithoutGetter();
List<BeanWithoutGetter> data = Arrays.asList(bean);
spark.createDataFrame(data, BeanWithoutGetter.class).show();
```

- Before

It throws an exception as below:

```
java.lang.NullPointerException
	at org.spark_project.guava.reflect.TypeToken.method(TypeToken.java:465)
	at org.apache.spark.sql.catalyst.JavaTypeInference$$anonfun$2.apply(JavaTypeInference.scala:126)
	at org.apache.spark.sql.catalyst.JavaTypeInference$$anonfun$2.apply(JavaTypeInference.scala:125)
```

- After

```
++
||
++
||
++
```

**Supports empty bean in encoder creation**

```java
public static class EmptyBean implements Serializable {}

EmptyBean bean = new EmptyBean();
List<EmptyBean> data = Arrays.asList(bean);
spark.createDataset(data, Encoders.bean(EmptyBean.class)).show();
```

- Before

throws an exception as below:

```
java.lang.UnsupportedOperationException: Cannot infer type for class EmptyBean because it is not bean-compliant
	at org.apache.spark.sql.catalyst.JavaTypeInference$.org$apache$spark$sql$catalyst$JavaTypeInference$$serializerFor(JavaTypeInference.scala:436)
	at org.apache.spark.sql.catalyst.JavaTypeInference$.serializerFor(JavaTypeInference.scala:341)
```

- After

```
++
||
++
||
++
```

## How was this patch tested?

Unit test in `JavaDataFrameSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17013 from HyukjinKwon/SPARK-19666.
2017-02-22 12:42:23 -08:00
Bogdan Raducanu 10c566cc3b [SPARK-13721][SQL] Make GeneratorOuter unresolved.
## What changes were proposed in this pull request?

This is a small change to make GeneratorOuter always unresolved. It is mostly no-op change but makes it more clear since GeneratorOuter shouldn't survive analysis phase.
This requires also handling in ResolveAliases rule.

## How was this patch tested?
Existing generator tests.

Author: Bogdan Raducanu <bogdan@databricks.com>
Author: Reynold Xin <rxin@databricks.com>

Closes #17026 from bogdanrdc/PR16958.
2017-02-22 15:42:40 +01:00
windpiger 65fe902e13 [SPARK-19598][SQL] Remove the alias parameter in UnresolvedRelation
## What changes were proposed in this pull request?

Remove the alias parameter in `UnresolvedRelation`, and use `SubqueryAlias` to replace it.
This can simplify some `match case` situations.

For example, the broadcast hint pull request can have one fewer case https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala#L57-L61

## How was this patch tested?
add some unit tests

Author: windpiger <songjun@outlook.com>

Closes #16956 from windpiger/removeUnresolveTableAlias.
2017-02-19 16:50:16 -08:00
Ala Luszczak b486ffc86d [SPARK-19447] Make Range operator generate "recordsRead" metric
## What changes were proposed in this pull request?

The Range was modified to produce "recordsRead" metric instead of "generated rows". The tests were updated and partially moved to SQLMetricsSuite.

## How was this patch tested?

Unit tests.

Author: Ala Luszczak <ala@databricks.com>

Closes #16960 from ala/range-records-read.
2017-02-18 07:51:41 -08:00
Nathan Howell 21fde57f15 [SPARK-18352][SQL] Support parsing multiline json files
## What changes were proposed in this pull request?

If a new option `wholeFile` is set to `true` the JSON reader will parse each file (instead of a single line) as a value. This is done with Jackson streaming and it should be capable of parsing very large documents, assuming the row will fit in memory.

Because the file is not buffered in memory the corrupt record handling is also slightly different when `wholeFile` is enabled: the corrupt column will contain the filename instead of the literal JSON if there is a parsing failure. It would be easy to extend this to add the parser location (line, column and byte offsets) to the output if desired.

These changes have allowed types other than `String` to be parsed. Support for `UTF8String` and `Text` have been added (alongside `String` and `InputFormat`) and no longer require a conversion to `String` just for parsing.

I've also included a few other changes that generate slightly better bytecode and (imo) make it more obvious when and where boxing is occurring in the parser. These are included as separate commits, let me know if they should be flattened into this PR or moved to a new one.

## How was this patch tested?

New and existing unit tests. No performance or load tests have been run.

Author: Nathan Howell <nhowell@godaddy.com>

Closes #16386 from NathanHowell/SPARK-18352.
2017-02-16 20:51:19 -08:00
Sean Owen 0e2405490f
[SPARK-19550][BUILD][CORE][WIP] Remove Java 7 support
- Move external/java8-tests tests into core, streaming, sql and remove
- Remove MaxPermGen and related options
- Fix some reflection / TODOs around Java 8+ methods
- Update doc references to 1.7/1.8 differences
- Remove Java 7/8 related build profiles
- Update some plugins for better Java 8 compatibility
- Fix a few Java-related warnings

For the future:

- Update Java 8 examples to fully use Java 8
- Update Java tests to use lambdas for simplicity
- Update Java internal implementations to use lambdas

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #16871 from srowen/SPARK-19493.
2017-02-16 12:32:45 +00:00
Tejas Patil f041e55eef [SPARK-19618][SQL] Inconsistency wrt max. buckets allowed from Dataframe API vs SQL
## What changes were proposed in this pull request?

Jira: https://issues.apache.org/jira/browse/SPARK-19618

Moved the check for validating number of buckets from `DataFrameWriter` to `BucketSpec` creation

## How was this patch tested?

- Added more unit tests

Author: Tejas Patil <tejasp@fb.com>

Closes #16948 from tejasapatil/SPARK-19618_max_buckets.
2017-02-15 22:45:58 -08:00
Takuya UESHIN 865b2fd84c [SPARK-18937][SQL] Timezone support in CSV/JSON parsing
## What changes were proposed in this pull request?

This is a follow-up pr of #16308.

This pr enables timezone support in CSV/JSON parsing.

We should introduce `timeZone` option for CSV/JSON datasources (the default value of the option is session local timezone).

The datasources should use the `timeZone` option to format/parse to write/read timestamp values.
Notice that while reading, if the timestampFormat has the timezone info, the timezone will not be used because we should respect the timezone in the values.

For example, if you have timestamp `"2016-01-01 00:00:00"` in `GMT`, the values written with the default timezone option, which is `"GMT"` because session local timezone is `"GMT"` here, are:

```scala
scala> spark.conf.set("spark.sql.session.timeZone", "GMT")

scala> val df = Seq(new java.sql.Timestamp(1451606400000L)).toDF("ts")
df: org.apache.spark.sql.DataFrame = [ts: timestamp]

scala> df.show()
+-------------------+
|ts                 |
+-------------------+
|2016-01-01 00:00:00|
+-------------------+

scala> df.write.json("/path/to/gmtjson")
```

```sh
$ cat /path/to/gmtjson/part-*
{"ts":"2016-01-01T00:00:00.000Z"}
```

whereas setting the option to `"PST"`, they are:

```scala
scala> df.write.option("timeZone", "PST").json("/path/to/pstjson")
```

```sh
$ cat /path/to/pstjson/part-*
{"ts":"2015-12-31T16:00:00.000-08:00"}
```

We can properly read these files even if the timezone option is wrong because the timestamp values have timezone info:

```scala
scala> val schema = new StructType().add("ts", TimestampType)
schema: org.apache.spark.sql.types.StructType = StructType(StructField(ts,TimestampType,true))

scala> spark.read.schema(schema).json("/path/to/gmtjson").show()
+-------------------+
|ts                 |
+-------------------+
|2016-01-01 00:00:00|
+-------------------+

scala> spark.read.schema(schema).option("timeZone", "PST").json("/path/to/gmtjson").show()
+-------------------+
|ts                 |
+-------------------+
|2016-01-01 00:00:00|
+-------------------+
```

And even if `timezoneFormat` doesn't contain timezone info, we can properly read the values with setting correct timezone option:

```scala
scala> df.write.option("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss").option("timeZone", "JST").json("/path/to/jstjson")
```

```sh
$ cat /path/to/jstjson/part-*
{"ts":"2016-01-01T09:00:00"}
```

```scala
// wrong result
scala> spark.read.schema(schema).option("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss").json("/path/to/jstjson").show()
+-------------------+
|ts                 |
+-------------------+
|2016-01-01 09:00:00|
+-------------------+

// correct result
scala> spark.read.schema(schema).option("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss").option("timeZone", "JST").json("/path/to/jstjson").show()
+-------------------+
|ts                 |
+-------------------+
|2016-01-01 00:00:00|
+-------------------+
```

This pr also makes `JsonToStruct` and `StructToJson` `TimeZoneAwareExpression` to be able to evaluate values with timezone option.

## How was this patch tested?

Existing tests and added some tests.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #16750 from ueshin/issues/SPARK-18937.
2017-02-15 13:26:34 -08:00
jiangxingbo 3755da76c3 [SPARK-19331][SQL][TESTS] Improve the test coverage of SQLViewSuite
Move `SQLViewSuite` from `sql/hive` to `sql/core`, so we can test the view supports without hive metastore. Also moved the test cases that specified to hive to `HiveSQLViewSuite`.

Improve the test coverage of SQLViewSuite, cover the following cases:
1. view resolution(possibly a referenced table/view have changed after the view creation);
2. handle a view with user specified column names;
3. improve the test cases for a nested view.

Also added a test case for cyclic view reference, which is a known issue that is not fixed yet.

N/A

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #16674 from jiangxb1987/view-test.
2017-02-15 10:47:11 -08:00
Liang-Chi Hsieh acf71c63cd [SPARK-16475][SQL] broadcast hint for SQL queries - disallow space as the delimiter
## What changes were proposed in this pull request?

A follow-up to disallow space as the delimiter in broadcast hint.

## How was this patch tested?

Jenkins test.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #16941 from viirya/disallow-space-delimiter.
2017-02-15 18:48:02 +01:00
Zhenhua Wang 601b9c3e68 [SPARK-17076][SQL] Cardinality estimation for join based on basic column statistics
## What changes were proposed in this pull request?

Support cardinality estimation and stats propagation for all join types.

Limitations:
- For inner/outer joins without any equal condition, we estimate it like cartesian product.
- For left semi/anti joins, since we can't apply the heuristics for inner join to it, for now we just propagate the statistics from left side. We should support them when other advanced stats (e.g. histograms) are available in spark.

## How was this patch tested?

Add a new test suite.

Author: Zhenhua Wang <wzh_zju@163.com>
Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #16228 from wzhfy/joinEstimate.
2017-02-15 08:21:51 -08:00
Reynold Xin 733c59ec1e [SPARK-16475][SQL] broadcast hint for SQL queries - follow up
## What changes were proposed in this pull request?
A small update to https://github.com/apache/spark/pull/16925

1. Rename SubstituteHints -> ResolveHints to be more consistent with rest of the rules.
2. Added more documentation in the rule and be more defensive / future proof to skip views as well as CTEs.

## How was this patch tested?
This pull request contains no real logic change and all behavior should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #16939 from rxin/SPARK-16475.
2017-02-15 17:10:49 +01:00
sureshthalamati f48c5a57d6 [SPARK-19318][SQL] Fix to treat JDBC connection properties specified by the user in case-sensitive manner.
## What changes were proposed in this pull request?
The reason for test failure is that the property “oracle.jdbc.mapDateToTimestamp” set by the test was getting converted into all lower case. Oracle database expects this property in case-sensitive manner.

This test was passing in previous releases because connection properties were sent as user specified for the test case scenario. Fixes to handle all option uniformly in case-insensitive manner, converted the JDBC connection properties also to lower case.

This PR  enhances CaseInsensitiveMap to keep track of input case-sensitive keys , and uses those when creating connection properties that are passed to the JDBC connection.

Alternative approach PR https://github.com/apache/spark/pull/16847  is to pass original input keys to JDBC data source by adding check in the  Data source class and handle case-insensitivity in the JDBC source code.

## How was this patch tested?
Added new test cases to JdbcSuite , and OracleIntegrationSuite. Ran docker integration tests passed on my laptop, all tests passed successfully.

Author: sureshthalamati <suresh.thalamati@gmail.com>

Closes #16891 from sureshthalamati/jdbc_case_senstivity_props_fix-SPARK-19318.
2017-02-14 15:34:12 -08:00
Reynold Xin da7aef7a0e [SPARK-16475][SQL] Broadcast hint for SQL Queries
## What changes were proposed in this pull request?
This pull request introduces a simple hint infrastructure to SQL and implements broadcast join hint using the infrastructure.

The hint syntax looks like the following:
```
SELECT /*+ BROADCAST(t) */ * FROM t
```

For broadcast hint, we accept "BROADCAST", "BROADCASTJOIN", and "MAPJOIN", and a sequence of relation aliases can be specified in the hint. A broadcast hint plan node will be inserted on top of any relation (that is not aliased differently), subquery, or common table expression that match the specified name.

The hint resolution works by recursively traversing down the query plan to find a relation or subquery that matches one of the specified broadcast aliases. The traversal does not go past beyond any existing broadcast hints, subquery aliases. This rule happens before common table expressions.

Note that there was an earlier patch in https://github.com/apache/spark/pull/14426. This is a rewrite of that patch, with different semantics and simpler test cases.

## How was this patch tested?
Added a new unit test suite for the broadcast hint rule (SubstituteHintsSuite) and new test cases for parser change (in PlanParserSuite). Also added end-to-end test case in BroadcastSuite.

Author: Reynold Xin <rxin@databricks.com>
Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #16925 from rxin/SPARK-16475-broadcast-hint.
2017-02-14 14:11:17 -08:00
ouyangxiaochen 6e45b547ce [SPARK-19115][SQL] Supporting Create Table Like Location
What changes were proposed in this pull request?

Support CREATE [EXTERNAL] TABLE LIKE LOCATION... syntax for Hive serde and datasource tables.
In this PR,we follow SparkSQL design rules :

    supporting create table like view or physical table or temporary view with location.
    creating a table with location,this table will be an external table other than managed table.

How was this patch tested?

Add new test cases and update existing test cases

Author: ouyangxiaochen <ou.yangxiaochen@zte.com.cn>

Closes #16868 from ouyangxiaochen/spark19115.
2017-02-13 19:41:44 -08:00
hyukjinkwon 9af8f743b0 [SPARK-19435][SQL] Type coercion between ArrayTypes
## What changes were proposed in this pull request?

This PR proposes to support type coercion between `ArrayType`s where the element types are compatible.

**Before**

```
Seq(Array(1)).toDF("a").selectExpr("greatest(a, array(1D))")
org.apache.spark.sql.AnalysisException: cannot resolve 'greatest(`a`, array(1.0D))' due to data type mismatch: The expressions should all have the same type, got GREATEST(array<int>, array<double>).; line 1 pos 0;

Seq(Array(1)).toDF("a").selectExpr("least(a, array(1D))")
org.apache.spark.sql.AnalysisException: cannot resolve 'least(`a`, array(1.0D))' due to data type mismatch: The expressions should all have the same type, got LEAST(array<int>, array<double>).; line 1 pos 0;

sql("SELECT * FROM values (array(0)), (array(1D)) as data(a)")
org.apache.spark.sql.AnalysisException: incompatible types found in column a for inline table; line 1 pos 14

Seq(Array(1)).toDF("a").union(Seq(Array(1D)).toDF("b"))
org.apache.spark.sql.AnalysisException: Union can only be performed on tables with the compatible column types. ArrayType(DoubleType,false) <> ArrayType(IntegerType,false) at the first column of the second table;;

sql("SELECT IF(1=1, array(1), array(1D))")
org.apache.spark.sql.AnalysisException: cannot resolve '(IF((1 = 1), array(1), array(1.0D)))' due to data type mismatch: differing types in '(IF((1 = 1), array(1), array(1.0D)))' (array<int> and array<double>).; line 1 pos 7;
```

**After**

```scala
Seq(Array(1)).toDF("a").selectExpr("greatest(a, array(1D))")
res5: org.apache.spark.sql.DataFrame = [greatest(a, array(1.0)): array<double>]

Seq(Array(1)).toDF("a").selectExpr("least(a, array(1D))")
res6: org.apache.spark.sql.DataFrame = [least(a, array(1.0)): array<double>]

sql("SELECT * FROM values (array(0)), (array(1D)) as data(a)")
res8: org.apache.spark.sql.DataFrame = [a: array<double>]

Seq(Array(1)).toDF("a").union(Seq(Array(1D)).toDF("b"))
res10: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [a: array<double>]

sql("SELECT IF(1=1, array(1), array(1D))")
res15: org.apache.spark.sql.DataFrame = [(IF((1 = 1), array(1), array(1.0))): array<double>]
```

## How was this patch tested?

Unit tests in `TypeCoercion` and Jenkins tests and

building with scala 2.10

```scala
./dev/change-scala-version.sh 2.10
./build/mvn -Pyarn -Phadoop-2.4 -Dscala-2.10 -DskipTests clean package
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16777 from HyukjinKwon/SPARK-19435.
2017-02-13 13:10:57 -08:00
hyukjinkwon 4321ff9edd [SPARK-19544][SQL] Improve error message when some column types are compatible and others are not in set operations
## What changes were proposed in this pull request?

This PR proposes to fix the error message when some data types are compatible and others are not in set/union operation.

Currently, the code below:

```scala
Seq((1,("a", 1))).toDF.union(Seq((1L,("a", "b"))).toDF)
```

throws an exception saying `LongType` and `IntegerType` are incompatible types. It should say something about `StructType`s with more readable format as below:

**Before**

```
Union can only be performed on tables with the compatible column types.
LongType <> IntegerType at the first column of the second table;;
```

**After**

```
Union can only be performed on tables with the compatible column types.
struct<_1:string,_2:string> <> struct<_1:string,_2:int> at the second column of the second table;;
```

*I manually inserted a newline in the messages above for readability only in this PR description.

## How was this patch tested?

Unit tests in `AnalysisErrorSuite`, manual tests and build wth Scala 2.10.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16882 from HyukjinKwon/SPARK-19544.
2017-02-13 16:08:31 +01:00
windpiger 04ad822534 [SPARK-19496][SQL] to_date udf to return null when input date is invalid
## What changes were proposed in this pull request?

Currently the udf  `to_date` has different return value with an invalid date input.

```
SELECT to_date('2015-07-22', 'yyyy-dd-MM') ->  return `2016-10-07`
SELECT to_date('2014-31-12')    -> return null
```

As discussed in JIRA [SPARK-19496](https://issues.apache.org/jira/browse/SPARK-19496), we should return null in both situations when the input date is invalid

## How was this patch tested?
unit test added

Author: windpiger <songjun@outlook.com>

Closes #16870 from windpiger/to_date.
2017-02-13 12:25:13 +01:00
Herman van Hovell de8a03e682 [SPARK-19459][SQL] Add Hive datatype (char/varchar) to StructField metadata
## What changes were proposed in this pull request?
Reading from an existing ORC table which contains `char` or `varchar` columns can fail with a `ClassCastException` if the table metadata has been created using Spark. This is caused by the fact that spark internally replaces `char` and `varchar` columns with a `string` column.

This PR fixes this by adding the hive type to the `StructField's` metadata under the `HIVE_TYPE_STRING` key. This is picked up by the `HiveClient` and the ORC reader, see https://github.com/apache/spark/pull/16060 for more details on how the metadata is used.

## How was this patch tested?
Added a regression test to `OrcSourceSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #16804 from hvanhovell/SPARK-19459.
2017-02-10 11:06:57 -08:00
Burak Yavuz d5593f7f57 [SPARK-19543] from_json fails when the input row is empty
## What changes were proposed in this pull request?

Using from_json on a column with an empty string results in: java.util.NoSuchElementException: head of empty list.

This is because `parser.parse(input)` may return `Nil` when `input.trim.isEmpty`

## How was this patch tested?

Regression test in `JsonExpressionsSuite`

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #16881 from brkyvz/json-fix.
2017-02-10 12:55:06 +01:00
jiangxingbo af63c52fd3 [SPARK-19025][SQL] Remove SQL builder for operators
## What changes were proposed in this pull request?

With the new approach of view resolution, we can get rid of SQL generation on view creation, so let's remove SQL builder for operators.

Note that, since all sql generation for operators is defined in one file (org.apache.spark.sql.catalyst.SQLBuilder), it’d be trivial to recover it in the future.

## How was this patch tested?

N/A

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #16869 from jiangxb1987/SQLBuilder.
2017-02-09 19:35:39 +01:00
Bogdan Raducanu 1af0dee418 [SPARK-19512][SQL] codegen for compare structs fails
## What changes were proposed in this pull request?

Set currentVars to null in GenerateOrdering.genComparisons before genCode is called. genCode ignores INPUT_ROW if currentVars is not null and in genComparisons we want it to use INPUT_ROW.

## How was this patch tested?

Added test with 2 queries in WholeStageCodegenSuite

Author: Bogdan Raducanu <bogdan.rdc@gmail.com>

Closes #16852 from bogdanrdc/SPARK-19512.
2017-02-09 19:15:11 +01:00
Ala Luszczak 4064574d03 [SPARK-19514] Making range interruptible.
## What changes were proposed in this pull request?

Previously range operator could not be interrupted. For example, using DAGScheduler.cancelStage(...) on a query with range might have been ineffective.

This change adds periodic checks of TaskContext.isInterrupted to codegen version, and InterruptibleOperator to non-codegen version.

I benchmarked the performance of codegen version on a sample query `spark.range(1000L * 1000 * 1000 * 10).count()` and there is no measurable difference.

## How was this patch tested?

Adds a unit test.

Author: Ala Luszczak <ala@databricks.com>

Closes #16872 from ala/SPARK-19514b.
2017-02-09 19:07:06 +01:00
Liwei Lin 9d9d67c795 [SPARK-19265][SQL][FOLLOW-UP] Configurable tableRelationCache maximum size
## What changes were proposed in this pull request?

SPARK-19265 had made table relation cache general; this follow-up aims to make `tableRelationCache`'s maximum size configurable.

In order to do sanity-check, this patch also adds a `checkValue()` method to `TypedConfigBuilder`.

## How was this patch tested?

new test case: `test("conf entry: checkValue()")`

Author: Liwei Lin <lwlin7@gmail.com>

Closes #16736 from lw-lin/conf.
2017-02-09 00:48:47 -05:00
Wenchen Fan 50a991264c [SPARK-19359][SQL] renaming partition should not leave useless directories
## What changes were proposed in this pull request?

Hive metastore is not case-preserving and keep partition columns with lower case names. If Spark SQL creates a table with upper-case partition column names using `HiveExternalCatalog`, when we rename partition, it first calls the HiveClient to renamePartition, which will create a new lower case partition path, then Spark SQL renames the lower case path to upper-case.

However, when we rename a nested path, different file systems have different behaviors. e.g. in jenkins, renaming `a=1/b=2` to `A=2/B=2` will success, but leave an empty directory `a=1`. in mac os, the renaming doesn't work as expected and result to `a=1/B=2`.

This PR renames the partition directory recursively from the first partition column in `HiveExternalCatalog`, to be most compatible with different file systems.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16837 from cloud-fan/partition.
2017-02-09 00:39:22 -05:00
gatorsmile 4d4d0de7f6 [SPARK-19279][SQL][FOLLOW-UP] Infer Schema for Hive Serde Tables
### What changes were proposed in this pull request?
`table.schema` is always not empty for partitioned tables, because `table.schema` also contains the partitioned columns, even if the original table does not have any column. This PR is to fix the issue.

### How was this patch tested?
Added a test case

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16848 from gatorsmile/inferHiveSerdeSchema.
2017-02-08 10:11:44 -05:00
Tathagata Das aeb80348dd [SPARK-19413][SS] MapGroupsWithState for arbitrary stateful operations
## What changes were proposed in this pull request?

`mapGroupsWithState` is a new API for arbitrary stateful operations in Structured Streaming, similar to `DStream.mapWithState`

*Requirements*
- Users should be able to specify a function that can do the following
- Access the input row corresponding to a key
- Access the previous state corresponding to a key
- Optionally, update or remove the state
- Output any number of new rows (or none at all)

*Proposed API*
```
// ------------ New methods on KeyValueGroupedDataset ------------
class KeyValueGroupedDataset[K, V] {
	// Scala friendly
	def mapGroupsWithState[S: Encoder, U: Encoder](func: (K, Iterator[V], KeyedState[S]) => U)
        def flatMapGroupsWithState[S: Encode, U: Encoder](func: (K, Iterator[V], KeyedState[S]) => Iterator[U])
	// Java friendly
       def mapGroupsWithState[S, U](func: MapGroupsWithStateFunction[K, V, S, R], stateEncoder: Encoder[S], resultEncoder: Encoder[U])
       def flatMapGroupsWithState[S, U](func: FlatMapGroupsWithStateFunction[K, V, S, R], stateEncoder: Encoder[S], resultEncoder: Encoder[U])
}

// ------------------- New Java-friendly function classes -------------------
public interface MapGroupsWithStateFunction<K, V, S, R> extends Serializable {
  R call(K key, Iterator<V> values, state: KeyedState<S>) throws Exception;
}
public interface FlatMapGroupsWithStateFunction<K, V, S, R> extends Serializable {
  Iterator<R> call(K key, Iterator<V> values, state: KeyedState<S>) throws Exception;
}

// ---------------------- Wrapper class for state data ----------------------
trait State[S] {
	def exists(): Boolean
  	def get(): S 			// throws Exception is state does not exist
	def getOption(): Option[S]
	def update(newState: S): Unit
	def remove(): Unit		// exists() will be false after this
}
```

Key Semantics of the State class
- The state can be null.
- If the state.remove() is called, then state.exists() will return false, and getOption will returm None.
- After that state.update(newState) is called, then state.exists() will return true, and getOption will return Some(...).
- None of the operations are thread-safe. This is to avoid memory barriers.

*Usage*
```
val stateFunc = (word: String, words: Iterator[String, runningCount: KeyedState[Long]) => {
    val newCount = words.size + runningCount.getOption.getOrElse(0L)
    runningCount.update(newCount)
   (word, newCount)
}

dataset					                        // type is Dataset[String]
  .groupByKey[String](w => w)        	                // generates KeyValueGroupedDataset[String, String]
  .mapGroupsWithState[Long, (String, Long)](stateFunc)	// returns Dataset[(String, Long)]
```

## How was this patch tested?
New unit tests.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #16758 from tdas/mapWithState.
2017-02-07 20:21:00 -08:00
Herman van Hovell 73ee73945e [SPARK-18609][SPARK-18841][SQL] Fix redundant Alias removal in the optimizer
## What changes were proposed in this pull request?
The optimizer tries to remove redundant alias only projections from the query plan using the `RemoveAliasOnlyProject` rule. The current rule identifies removes such a project and rewrites the project's attributes in the **entire** tree. This causes problems when parts of the tree are duplicated (for instance a self join on a temporary view/CTE)  and the duplicated part contains the alias only project, in this case the rewrite will break the tree.

This PR fixes these problems by using a blacklist for attributes that are not to be moved, and by making sure that attribute remapping is only done for the parent tree, and not for unrelated parts of the query plan.

The current tree transformation infrastructure works very well if the transformation at hand requires little or a global contextual information. In this case we need to know both the attributes that were not to be moved, and we also needed to know which child attributes were modified. This cannot be done easily using the current infrastructure, and solutions typically involves transversing the query plan multiple times (which is super slow). I have moved around some code in `TreeNode`, `QueryPlan` and `LogicalPlan`to make this much more straightforward; this basically allows you to manually traverse the tree.

This PR subsumes the following PRs by windpiger:
Closes https://github.com/apache/spark/pull/16267
Closes https://github.com/apache/spark/pull/16255

## How was this patch tested?
I have added unit tests to `RemoveRedundantAliasAndProjectSuite` and I have added integration tests to the `SQLQueryTestSuite.union` and `SQLQueryTestSuite.cte` test cases.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #16757 from hvanhovell/SPARK-18609.
2017-02-07 22:28:59 +01:00
anabranch 7a7ce272fe [SPARK-16609] Add to_date/to_timestamp with format functions
## What changes were proposed in this pull request?

This pull request adds two new user facing functions:
- `to_date` which accepts an expression and a format and returns a date.
- `to_timestamp` which accepts an expression and a format and returns a timestamp.

For example, Given a date in format: `2016-21-05`. (YYYY-dd-MM)

### Date Function
*Previously*
```
to_date(unix_timestamp(lit("2016-21-05"), "yyyy-dd-MM").cast("timestamp"))
```
*Current*
```
to_date(lit("2016-21-05"), "yyyy-dd-MM")
```

### Timestamp Function
*Previously*
```
unix_timestamp(lit("2016-21-05"), "yyyy-dd-MM").cast("timestamp")
```
*Current*
```
to_timestamp(lit("2016-21-05"), "yyyy-dd-MM")
```
### Tasks

- [X] Add `to_date` to Scala Functions
- [x] Add `to_date` to Python Functions
- [x] Add `to_date` to SQL Functions
- [X] Add `to_timestamp` to Scala Functions
- [x] Add `to_timestamp` to Python Functions
- [x] Add `to_timestamp` to SQL Functions
- [x] Add function to R

## How was this patch tested?

- [x] Add Functions to `DateFunctionsSuite`
- Test new `ParseToTimestamp` Expression (*not necessary*)
- Test new `ParseToDate` Expression (*not necessary*)
- [x] Add test for R
- [x] Add test for Python in test.py

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: anabranch <wac.chambers@gmail.com>
Author: Bill Chambers <bill@databricks.com>
Author: anabranch <bill@databricks.com>

Closes #16138 from anabranch/SPARK-16609.
2017-02-07 15:50:30 +01:00
gagan taneja e99e34d0f3 [SPARK-19118][SQL] Percentile support for frequency distribution table
## What changes were proposed in this pull request?

I have a frequency distribution table with following entries
Age,    No of person
21, 10
22, 15
23, 18
..
..
30, 14
Moreover it is common to have data in frequency distribution format to further calculate Percentile, Median. With current implementation
It would be very difficult and complex to find the percentile.
Therefore i am proposing enhancement to current Percentile and Approx Percentile implementation to take frequency distribution column into consideration

## How was this patch tested?
1) Enhanced /sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala to cover the additional functionality
2) Run some performance benchmark test with 20 million row in local environment and did not see any performance degradation

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: gagan taneja <tanejagagan@gagans-MacBook-Pro.local>

Closes #16497 from tanejagagan/branch-18940.
2017-02-07 14:05:22 +01:00
Eyal Farago a97edc2cf4 [SPARK-18601][SQL] Simplify Create/Get complex expression pairs in optimizer
## What changes were proposed in this pull request?
It often happens that a complex object (struct/map/array) is created only to get elements from it in an subsequent expression. We can add an optimizer rule for this.

## How was this patch tested?
unit-tests

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Eyal Farago <eyal@nrgene.com>
Author: eyal farago <eyal.farago@gmail.com>

Closes #16043 from eyalfa/SPARK-18601.
2017-02-07 10:54:55 +01:00
gatorsmile d6dc603ed4 [SPARK-19441][SQL] Remove IN type coercion from PromoteStrings
### What changes were proposed in this pull request?
The removed codes for `IN` are not reachable, because the previous rule `InConversion` already resolves the type coercion issues.

### How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16783 from gatorsmile/typeCoercionIn.
2017-02-07 09:59:16 +08:00
Herman van Hovell cb2677b860 [SPARK-19472][SQL] Parser should not mistake CASE WHEN(...) for a function call
## What changes were proposed in this pull request?
The SQL parser can mistake a `WHEN (...)` used in `CASE` for a function call. This happens in cases like the following:
```sql
select case when (1) + case when 1 > 0 then 1 else 0 end = 2 then 1 else 0 end
from tb
```
This PR fixes this by re-organizing the case related parsing rules.

## How was this patch tested?
Added a regression test to the `ExpressionParserSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #16821 from hvanhovell/SPARK-19472.
2017-02-06 15:28:13 -05:00
Wenchen Fan aff53021cf [SPARK-19080][SQL] simplify data source analysis
## What changes were proposed in this pull request?

The current way of resolving `InsertIntoTable` and `CreateTable` is convoluted: sometimes we replace them with concrete implementation commands during analysis, sometimes during planning phase.

And the error checking logic is also a mess: we may put it in extended analyzer rules, or extended checking rules, or `CheckAnalysis`.

This PR simplifies the data source analysis:

1.  `InsertIntoTable` and `CreateTable` are always unresolved and need to be replaced by concrete implementation commands during analysis.
2. The error checking logic is mainly in 2 rules: `PreprocessTableCreation` and `PreprocessTableInsertion`.

## How was this patch tested?

existing test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16269 from cloud-fan/ddl.
2017-02-07 00:36:57 +08:00
Liang-Chi Hsieh 0674e7eb85 [SPARK-19425][SQL] Make ExtractEquiJoinKeys support UDT columns
## What changes were proposed in this pull request?

DataFrame.except doesn't work for UDT columns. It is because `ExtractEquiJoinKeys` will run `Literal.default` against UDT. However, we don't handle UDT in `Literal.default` and an exception will throw like:

    java.lang.RuntimeException: no default for type
    org.apache.spark.ml.linalg.VectorUDT3bfc3ba7
      at org.apache.spark.sql.catalyst.expressions.Literal$.default(literals.scala:179)
      at org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys$$anonfun$4.apply(patterns.scala:117)
      at org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys$$anonfun$4.apply(patterns.scala:110)

More simple fix is just let `Literal.default` handle UDT by its sql type. So we can use more efficient join type on UDT.

Besides `except`, this also fixes other similar scenarios, so in summary this fixes:

* `except` on two Datasets with UDT
* `intersect` on two Datasets with UDT
* `Join` with the join conditions using `<=>` on UDT columns

## How was this patch tested?

Jenkins tests.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #16765 from viirya/df-except-for-udt.
2017-02-04 15:57:56 -08:00
hyukjinkwon 2f3c20bbdd [SPARK-19446][SQL] Remove unused findTightestCommonType in TypeCoercion
## What changes were proposed in this pull request?

This PR proposes to

- remove unused `findTightestCommonType` in `TypeCoercion` as suggested in https://github.com/apache/spark/pull/16777#discussion_r99283834
- rename `findTightestCommonTypeOfTwo ` to `findTightestCommonType`.
- fix comments accordingly

The usage was removed while refactoring/fixing in several JIRAs such as SPARK-16714, SPARK-16735 and SPARK-16646

## How was this patch tested?

Existing tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16786 from HyukjinKwon/SPARK-19446.
2017-02-03 22:10:17 -08:00
Dongjoon Hyun 52d4f61941 [SPARK-18909][SQL] The error messages in ExpressionEncoder.toRow/fromRow are too verbose
## What changes were proposed in this pull request?

In `ExpressionEncoder.toRow` and `fromRow`, we catch the exception and output `treeString` of serializer/deserializer expressions in the error message. However, encoder can be very complex and the serializer/deserializer expressions can be very large trees and blow up the log files(e.g. generate over 500mb logs for this single error message.) As a first attempt, this PR try to use `simpleString` instead.

**BEFORE**

```scala
scala> :paste
// Entering paste mode (ctrl-D to finish)

case class TestCaseClass(value: Int)
import spark.implicits._
Seq(TestCaseClass(1)).toDS().collect()

// Exiting paste mode, now interpreting.

java.lang.RuntimeException: Error while decoding: java.lang.NullPointerException
newInstance(class TestCaseClass)
+- assertnotnull(input[0, int, false], - field (class: "scala.Int", name: "value"), - root class: "TestCaseClass")
   +- input[0, int, false]

  at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.fromRow(ExpressionEncoder.scala:303)
...
```

**AFTER**

```scala
...
// Exiting paste mode, now interpreting.

java.lang.RuntimeException: Error while decoding: java.lang.NullPointerException
newInstance(class TestCaseClass)
  at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.fromRow(ExpressionEncoder.scala:303)
...
```

## How was this patch tested?

Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #16701 from dongjoon-hyun/SPARK-18909-EXPR-ERROR.
2017-02-03 20:26:53 +08:00
Liang-Chi Hsieh bf493686eb [SPARK-19411][SQL] Remove the metadata used to mark optional columns in merged Parquet schema for filter predicate pushdown
## What changes were proposed in this pull request?

There is a metadata introduced before to mark the optional columns in merged Parquet schema for filter predicate pushdown. As we upgrade to Parquet 1.8.2 which includes the fix for the pushdown of optional columns, we don't need this metadata now.

## How was this patch tested?

Jenkins tests.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #16756 from viirya/remove-optional-metadata.
2017-02-03 11:58:42 +01:00
hyukjinkwon f1a1f2607d
[SPARK-19402][DOCS] Support LaTex inline formula correctly and fix warnings in Scala/Java APIs generation
## What changes were proposed in this pull request?

This PR proposes three things as below:

- Support LaTex inline-formula, `\( ... \)` in Scala API documentation
  It seems currently,

  ```
  \( ... \)
  ```

  are rendered as they are, for example,

  <img width="345" alt="2017-01-30 10 01 13" src="https://cloud.githubusercontent.com/assets/6477701/22423960/ab37d54a-e737-11e6-9196-4f6229c0189c.png">

  It seems mistakenly more backslashes were added.

- Fix warnings Scaladoc/Javadoc generation
  This PR fixes t two types of warnings as below:

  ```
  [warn] .../spark/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala:335: Could not find any member to link for "UnsupportedOperationException".
  [warn]   /**
  [warn]   ^
  ```

  ```
  [warn] .../spark/sql/core/src/main/scala/org/apache/spark/sql/internal/VariableSubstitution.scala:24: Variable var undefined in comment for class VariableSubstitution in class VariableSubstitution
  [warn]  * `${var}`, `${system:var}` and `${env:var}`.
  [warn]      ^
  ```

- Fix Javadoc8 break
  ```
  [error] .../spark/mllib/target/java/org/apache/spark/ml/PredictionModel.java:7: error: reference not found
  [error]  *                       E.g., {link VectorUDT} for vector features.
  [error]                                       ^
  [error] .../spark/mllib/target/java/org/apache/spark/ml/PredictorParams.java:12: error: reference not found
  [error]    *                          E.g., {link VectorUDT} for vector features.
  [error]                                            ^
  [error] .../spark/mllib/target/java/org/apache/spark/ml/Predictor.java:10: error: reference not found
  [error]  *                       E.g., {link VectorUDT} for vector features.
  [error]                                       ^
  [error] .../spark/sql/hive/target/java/org/apache/spark/sql/hive/HiveAnalysis.java:5: error: reference not found
  [error]  * Note that, this rule must be run after {link PreprocessTableInsertion}.
  [error]                                                  ^
  ```

## How was this patch tested?

Manually via `sbt unidoc` and `jeykil build`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16741 from HyukjinKwon/warn-and-break.
2017-02-01 13:26:16 +00:00
gatorsmile f9156d2956 [SPARK-19406][SQL] Fix function to_json to respect user-provided options
### What changes were proposed in this pull request?
Currently, the function `to_json` allows users to provide options for generating JSON. However, it does not pass it to `JacksonGenerator`. Thus, it ignores the user-provided options. This PR is to fix it. Below is an example.

```Scala
val df = Seq(Tuple1(Tuple1(java.sql.Timestamp.valueOf("2015-08-26 18:00:00.0")))).toDF("a")
val options = Map("timestampFormat" -> "dd/MM/yyyy HH:mm")
df.select(to_json($"a", options)).show(false)
```
The current output is like
```
+--------------------------------------+
|structtojson(a)                       |
+--------------------------------------+
|{"_1":"2015-08-26T18:00:00.000-07:00"}|
+--------------------------------------+
```

After the fix, the output is like
```
+-------------------------+
|structtojson(a)          |
+-------------------------+
|{"_1":"26/08/2015 18:00"}|
+-------------------------+
```
### How was this patch tested?
Added test cases for both `from_json` and `to_json`

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16745 from gatorsmile/toJson.
2017-01-30 18:38:14 -08:00
Liwei Lin ade075aed4 [SPARK-19385][SQL] During canonicalization, NOT(...(l, r)) should not expect such cases that l.hashcode > r.hashcode
## What changes were proposed in this pull request?

During canonicalization, `NOT(...(l, r))` should not expect such cases that `l.hashcode > r.hashcode`.

Take the rule `case NOT(GreaterThan(l, r)) if l.hashcode > r.hashcode` for example, it should never be matched since `GreaterThan(l, r)` itself would be re-written as `GreaterThan(r, l)` given `l.hashcode > r.hashcode` after canonicalization.

This patch consolidates rules like `case NOT(GreaterThan(l, r)) if l.hashcode > r.hashcode` and `case NOT(GreaterThan(l, r))`.

## How was this patch tested?

This patch expanded the `NOT` test case to cover both cases where:
- `l.hashcode > r.hashcode`
- `l.hashcode < r.hashcode`

Author: Liwei Lin <lwlin7@gmail.com>

Closes #16719 from lw-lin/canonicalize.
2017-01-29 13:00:50 -08:00
hyukjinkwon 4e35c5a3d3
[SPARK-12970][DOCS] Fix the example in SturctType APIs for Scala and Java
## What changes were proposed in this pull request?

This PR fixes both,

javadoc8 break

```
[error] .../spark/sql/hive/target/java/org/apache/spark/sql/hive/FindHiveSerdeTable.java:3: error: reference not found
[error]  * Replaces {link SimpleCatalogRelation} with {link MetastoreRelation} if its table provider is hive.
```

and the example in `StructType` as a self-contained example as below:

```scala
import org.apache.spark.sql._
import org.apache.spark.sql.types._

val struct =
  StructType(
    StructField("a", IntegerType, true) ::
    StructField("b", LongType, false) ::
    StructField("c", BooleanType, false) :: Nil)

// Extract a single StructField.
val singleField = struct("b")
// singleField: StructField = StructField(b,LongType,false)

// If this struct does not have a field called "d", it throws an exception.
struct("d")
// java.lang.IllegalArgumentException: Field "d" does not exist.
//   ...

// Extract multiple StructFields. Field names are provided in a set.
// A StructType object will be returned.
val twoFields = struct(Set("b", "c"))
// twoFields: StructType =
//   StructType(StructField(b,LongType,false), StructField(c,BooleanType,false))

// Any names without matching fields will throw an exception.
// For the case shown below, an exception is thrown due to "d".
struct(Set("b", "c", "d"))
// java.lang.IllegalArgumentException: Field "d" does not exist.
//    ...
```

```scala
import org.apache.spark.sql._
import org.apache.spark.sql.types._

val innerStruct =
  StructType(
    StructField("f1", IntegerType, true) ::
    StructField("f2", LongType, false) ::
    StructField("f3", BooleanType, false) :: Nil)

val struct = StructType(
  StructField("a", innerStruct, true) :: Nil)

// Create a Row with the schema defined by struct
val row = Row(Row(1, 2, true))
```

Also, now when the column is missing, it throws an exception rather than ignoring.

## How was this patch tested?

Manually via `sbt unidoc`.

- Scaladoc

  <img width="665" alt="2017-01-26 12 54 13" src="https://cloud.githubusercontent.com/assets/6477701/22297905/1245620e-e362-11e6-9e22-43bb8d9871af.png">

- Javadoc

  <img width="722" alt="2017-01-26 12 54 27" src="https://cloud.githubusercontent.com/assets/6477701/22297899/0fd87e0c-e362-11e6-9033-7590bda1aea6.png">

  <img width="702" alt="2017-01-26 12 54 32" src="https://cloud.githubusercontent.com/assets/6477701/22297900/0fe14154-e362-11e6-9882-768381c53163.png">

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16703 from HyukjinKwon/SPARK-12970.
2017-01-27 10:06:54 +00:00
Takeshi YAMAMURO 9f523d3192 [SPARK-19338][SQL] Add UDF names in explain
## What changes were proposed in this pull request?
This pr added a variable for a UDF name in `ScalaUDF`.
Then, if the variable filled, `DataFrame#explain` prints the name.

## How was this patch tested?
Added a test in `UDFSuite`.

Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes #16707 from maropu/SPARK-19338.
2017-01-26 09:50:42 -08:00
Takuya UESHIN 2969fb4370 [SPARK-18936][SQL] Infrastructure for session local timezone support.
## What changes were proposed in this pull request?

As of Spark 2.1, Spark SQL assumes the machine timezone for datetime manipulation, which is bad if users are not in the same timezones as the machines, or if different users have different timezones.

We should introduce a session local timezone setting that is used for execution.

An explicit non-goal is locale handling.

### Semantics

Setting the session local timezone means that the timezone-aware expressions listed below should use the timezone to evaluate values, and also it should be used to convert (cast) between string and timestamp or between timestamp and date.

- `CurrentDate`
- `CurrentBatchTimestamp`
- `Hour`
- `Minute`
- `Second`
- `DateFormatClass`
- `ToUnixTimestamp`
- `UnixTimestamp`
- `FromUnixTime`

and below are implicitly timezone-aware through cast from timestamp to date:

- `DayOfYear`
- `Year`
- `Quarter`
- `Month`
- `DayOfMonth`
- `WeekOfYear`
- `LastDay`
- `NextDay`
- `TruncDate`

For example, if you have timestamp `"2016-01-01 00:00:00"` in `GMT`, the values evaluated by some of timezone-aware expressions are:

```scala
scala> val df = Seq(new java.sql.Timestamp(1451606400000L)).toDF("ts")
df: org.apache.spark.sql.DataFrame = [ts: timestamp]

scala> df.selectExpr("cast(ts as string)", "year(ts)", "month(ts)", "dayofmonth(ts)", "hour(ts)", "minute(ts)", "second(ts)").show(truncate = false)
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
|ts                 |year(CAST(ts AS DATE))|month(CAST(ts AS DATE))|dayofmonth(CAST(ts AS DATE))|hour(ts)|minute(ts)|second(ts)|
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
|2016-01-01 00:00:00|2016                  |1                      |1                           |0       |0         |0         |
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
```

whereas setting the session local timezone to `"PST"`, they are:

```scala
scala> spark.conf.set("spark.sql.session.timeZone", "PST")

scala> df.selectExpr("cast(ts as string)", "year(ts)", "month(ts)", "dayofmonth(ts)", "hour(ts)", "minute(ts)", "second(ts)").show(truncate = false)
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
|ts                 |year(CAST(ts AS DATE))|month(CAST(ts AS DATE))|dayofmonth(CAST(ts AS DATE))|hour(ts)|minute(ts)|second(ts)|
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
|2015-12-31 16:00:00|2015                  |12                     |31                          |16      |0         |0         |
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
```

Notice that even if you set the session local timezone, it affects only in `DataFrame` operations, neither in `Dataset` operations, `RDD` operations nor in `ScalaUDF`s. You need to properly handle timezone by yourself.

### Design of the fix

I introduced an analyzer to pass session local timezone to timezone-aware expressions and modified DateTimeUtils to take the timezone argument.

## How was this patch tested?

Existing tests and added tests for timezone aware expressions.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #16308 from ueshin/issues/SPARK-18350.
2017-01-26 11:51:05 +01:00
gmoehler f6480b1467 [SPARK-19311][SQL] fix UDT hierarchy issue
## What changes were proposed in this pull request?
acceptType() in UDT will no only accept the same type but also all base types

## How was this patch tested?
Manual test using a set of generated UDTs fixing acceptType() in my user defined types

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: gmoehler <moehler@de.ibm.com>

Closes #16660 from gmoehler/master.
2017-01-25 08:17:24 -08:00
Nattavut Sutyanyong f1ddca5fcc [SPARK-18863][SQL] Output non-aggregate expressions without GROUP BY in a subquery does not yield an error
## What changes were proposed in this pull request?
This PR will report proper error messages when a subquery expression contain an invalid plan. This problem is fixed by calling CheckAnalysis for the plan inside a subquery.

## How was this patch tested?
Existing tests and two new test cases on 2 forms of subquery, namely, scalar subquery and in/exists subquery.

````
-- TC 01.01
-- The column t2b in the SELECT of the subquery is invalid
-- because it is neither an aggregate function nor a GROUP BY column.
select t1a, t2b
from   t1, t2
where  t1b = t2c
and    t2b = (select max(avg)
              from   (select   t2b, avg(t2b) avg
                      from     t2
                      where    t2a = t1.t1b
                     )
             )
;

-- TC 01.02
-- Invalid due to the column t2b not part of the output from table t2.
select *
from   t1
where  t1a in (select   min(t2a)
               from     t2
               group by t2c
               having   t2c in (select   max(t3c)
                                from     t3
                                group by t3b
                                having   t3b > t2b ))
;
````

Author: Nattavut Sutyanyong <nsy.can@gmail.com>

Closes #16572 from nsyca/18863.
2017-01-25 17:04:36 +01:00
Nattavut Sutyanyong cdb691eb4d [SPARK-19017][SQL] NOT IN subquery with more than one column may return incorrect results
## What changes were proposed in this pull request?

This PR fixes the code in Optimizer phase where the NULL-aware expression of a NOT IN query is expanded in Rule `RewritePredicateSubquery`.

Example:
The query

 select a1,b1
 from   t1
 where  (a1,b1) not in (select a2,b2
                        from   t2);

has the (a1, b1) = (a2, b2) rewritten from (before this fix):

Join LeftAnti, ((isnull((_1#2 = a2#16)) || isnull((_2#3 = b2#17))) || ((_1#2 = a2#16) && (_2#3 = b2#17)))

to (after this fix):

Join LeftAnti, (((_1#2 = a2#16) || isnull((_1#2 = a2#16))) && ((_2#3 = b2#17) || isnull((_2#3 = b2#17))))

## How was this patch tested?

sql/test, catalyst/test and new test cases in SQLQueryTestSuite.

Author: Nattavut Sutyanyong <nsy.can@gmail.com>

Closes #16467 from nsyca/19017.
2017-01-24 23:31:06 +01:00
Wenchen Fan 59c184e028 [SPARK-17913][SQL] compare atomic and string type column may return confusing result
## What changes were proposed in this pull request?

Spark SQL follows MySQL to do the implicit type conversion for binary comparison: http://dev.mysql.com/doc/refman/5.7/en/type-conversion.html

However, this may return confusing result, e.g. `1 = 'true'` will return true, `19157170390056973L = '19157170390056971'` will return true.

I think it's more reasonable to follow postgres in this case, i.e. cast string to the type of the other side, but return null if the string is not castable to keep hive compatibility.

## How was this patch tested?

newly added tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15880 from cloud-fan/compare.
2017-01-24 10:18:25 -08:00
windpiger 752502be05 [SPARK-19246][SQL] CataLogTable's partitionSchema order and exist check
## What changes were proposed in this pull request?

CataLogTable's partitionSchema should check if each column name in partitionColumnNames must match one and only one field in schema, if not we should throw an exception

and CataLogTable's partitionSchema should keep order with partitionColumnNames

## How was this patch tested?
N/A

Author: windpiger <songjun@outlook.com>

Closes #16606 from windpiger/checkPartionColNameWithSchema.
2017-01-24 20:49:23 +08:00
jiangxingbo 3bdf3ee860 [SPARK-19272][SQL] Remove the param viewOriginalText from CatalogTable
## What changes were proposed in this pull request?

Hive will expand the view text, so it needs 2 fields: originalText and viewText. Since we don't expand the view text, but only add table properties, perhaps only a single field `viewText` is enough in CatalogTable.

This PR brought in the following changes:
1. Remove the param `viewOriginalText` from `CatalogTable`;
2. Update the output of command `DescribeTableCommand`.

## How was this patch tested?

Tested by exsiting test cases, also updated the failed test cases.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #16679 from jiangxb1987/catalogTable.
2017-01-24 12:37:30 +08:00
Wenchen Fan fcfd5d0bba [SPARK-19290][SQL] add a new extending interface in Analyzer for post-hoc resolution
## What changes were proposed in this pull request?

To implement DDL commands, we added several analyzer rules in sql/hive module to analyze DDL related plans. However, our `Analyzer` currently only have one extending interface: `extendedResolutionRules`, which defines extra rules that will be run together with other rules in the resolution batch, and doesn't fit DDL rules well, because:

1. DDL rules may do some checking and normalization, but we may do it many times as the resolution batch will run rules again and again, until fixed point, and it's hard to tell if a DDL rule has already done its checking and normalization. It's fine because DDL rules are idempotent, but it's bad for analysis performance
2. some DDL rules may depend on others, and it's pretty hard to write `if` conditions to guarantee the dependencies. It will be good if we have a batch which run rules in one pass, so that we can guarantee the dependencies by rules order.

This PR adds a new extending interface in `Analyzer`: `postHocResolutionRules`, which defines rules that will be run only once in a batch runs right after the resolution batch.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16645 from cloud-fan/analyzer.
2017-01-23 20:01:10 -08:00
Wenchen Fan de6ad3dfa7 [SPARK-19309][SQL] disable common subexpression elimination for conditional expressions
## What changes were proposed in this pull request?

As I pointed out in https://github.com/apache/spark/pull/15807#issuecomment-259143655 , the current subexpression elimination framework has a problem, it always evaluates all common subexpressions at the beginning, even they are inside conditional expressions and may not be accessed.

Ideally we should implement it like scala lazy val, so we only evaluate it when it gets accessed at lease once. https://github.com/apache/spark/issues/15837 tries this approach, but it seems too complicated and may introduce performance regression.

This PR simply stops common subexpression elimination for conditional expressions, with some cleanup.

## How was this patch tested?

regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16659 from cloud-fan/codegen.
2017-01-23 13:31:26 +08:00
gatorsmile 772035e771 [SPARK-19229][SQL] Disallow Creating Hive Source Tables when Hive Support is Not Enabled
### What changes were proposed in this pull request?
It is weird to create Hive source tables when using InMemoryCatalog. We are unable to operate it. This PR is to block users to create Hive source tables.

### How was this patch tested?
Fixed the test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16587 from gatorsmile/blockHiveTable.
2017-01-22 20:37:37 -08:00
Davies Liu 9b7a03f15a [SPARK-18589][SQL] Fix Python UDF accessing attributes from both side of join
## What changes were proposed in this pull request?

PythonUDF is unevaluable, which can not be used inside a join condition, currently the optimizer will push a PythonUDF which accessing both side of join into the join condition, then the query will fail to plan.

This PR fix this issue by checking the expression is evaluable  or not before pushing it into Join.

## How was this patch tested?

Add a regression test.

Author: Davies Liu <davies@databricks.com>

Closes #16581 from davies/pyudf_join.
2017-01-20 16:11:40 -08:00
Tathagata Das 552e5f0884 [SPARK-19314][SS][CATALYST] Do not allow sort before aggregation in Structured Streaming plan
## What changes were proposed in this pull request?

Sort in a streaming plan should be allowed only after a aggregation in complete mode. Currently it is incorrectly allowed when present anywhere in the plan. It gives unpredictable potentially incorrect results.

## How was this patch tested?
New test

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #16662 from tdas/SPARK-19314.
2017-01-20 14:04:51 -08:00
wangzhenhua 039ed9fe8a [SPARK-19271][SQL] Change non-cbo estimation of aggregate
## What changes were proposed in this pull request?

Change non-cbo estimation behavior of aggregate:
- If groupExpression is empty, we can know row count (=1) and the corresponding size;
- otherwise, estimation falls back to UnaryNode's computeStats method, which should not propagate rowCount and attributeStats in Statistics because they are not estimated in that method.

## How was this patch tested?

Added test case

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #16631 from wzhfy/aggNoCbo.
2017-01-19 22:18:47 -08:00
Wenchen Fan 2e62560024 [SPARK-19265][SQL] make table relation cache general and does not depend on hive
## What changes were proposed in this pull request?

We have a table relation plan cache in `HiveMetastoreCatalog`, which caches a lot of things: file status, resolved data source, inferred schema, etc.

However, it doesn't make sense to limit this cache with hive support, we should move it to SQL core module so that users can use this cache without hive support.

It can also reduce the size of `HiveMetastoreCatalog`, so that it's easier to remove it eventually.

main changes:
1. move the table relation cache to `SessionCatalog`
2. `SessionCatalog.lookupRelation` will return `SimpleCatalogRelation` and the analyzer will convert it to `LogicalRelation` or `MetastoreRelation` later, then `HiveSessionCatalog` doesn't need to override `lookupRelation` anymore
3. `FindDataSourceTable` will read/write the table relation cache.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16621 from cloud-fan/plan-cache.
2017-01-19 00:07:48 -08:00
jiangxingbo f85f29608d [SPARK-19024][SQL] Implement new approach to write a permanent view
## What changes were proposed in this pull request?

On CREATE/ALTER a view, it's no longer needed to generate a SQL text string from the LogicalPlan, instead we store the SQL query text、the output column names of the query plan, and current database to CatalogTable. Permanent views created by this approach can be resolved by current view resolution approach.

The main advantage includes:
1. If you update an underlying view, the current view also gets updated;
2. That gives us a change to get ride of SQL generation for operators.

Major changes of this PR:
1. Generate the view-specific properties(e.g. view default database, view query output column names) during permanent view creation and store them as properties in the CatalogTable;
2. Update the commands `CreateViewCommand` and `AlterViewAsCommand`, get rid of SQL generation from them.

## How was this patch tested?
Existing tests.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #16613 from jiangxb1987/view-write-path.
2017-01-18 19:13:01 +08:00
uncleGen eefdf9f9dd
[SPARK-19227][SPARK-19251] remove unused imports and outdated comments
## What changes were proposed in this pull request?
remove ununsed imports and outdated comments, and fix some minor code style issue.

## How was this patch tested?
existing ut

Author: uncleGen <hustyugm@gmail.com>

Closes #16591 from uncleGen/SPARK-19227.
2017-01-18 09:44:32 +00:00
Bogdan Raducanu 2992a0e79e [SPARK-13721][SQL] Support outer generators in DataFrame API
## What changes were proposed in this pull request?

Added outer_explode, outer_posexplode, outer_inline functions and expressions.
Some bug fixing in GenerateExec.scala for CollectionGenerator. Previously it was not correctly handling the case of outer with empty collections, only with nulls.

## How was this patch tested?

New tests added to GeneratorFunctionSuite

Author: Bogdan Raducanu <bogdan.rdc@gmail.com>

Closes #16608 from bogdanrdc/SPARK-13721.
2017-01-17 15:39:24 -08:00
jiangxingbo fee20df143 [MINOR][SQL] Remove duplicate call of reset() function in CurrentOrigin.withOrigin()
## What changes were proposed in this pull request?

Remove duplicate call of reset() function in CurrentOrigin.withOrigin().

## How was this patch tested?

Existing test cases.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #16615 from jiangxb1987/dummy-code.
2017-01-17 10:47:46 -08:00
gatorsmile a23debd7bc [SPARK-19129][SQL] SessionCatalog: Disallow empty part col values in partition spec
### What changes were proposed in this pull request?
Empty partition column values are not valid for partition specification. Before this PR, we accept users to do it; however, Hive metastore does not detect and disallow it too. Thus, users hit the following strange error.

```Scala
val df = spark.createDataFrame(Seq((0, "a"), (1, "b"))).toDF("partCol1", "name")
df.write.mode("overwrite").partitionBy("partCol1").saveAsTable("partitionedTable")
spark.sql("alter table partitionedTable drop partition(partCol1='')")
spark.table("partitionedTable").show()
```

In the above example, the WHOLE table is DROPPED when users specify a partition spec containing only one partition column with empty values.

When the partition columns contains more than one, Hive metastore APIs simply ignore the columns with empty values and treat it as partial spec. This is also not expected. This does not follow the actual Hive behaviors. This PR is to disallow users to specify such an invalid partition spec in the `SessionCatalog` APIs.

### How was this patch tested?
Added test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16583 from gatorsmile/disallowEmptyPartColValue.
2017-01-18 02:01:30 +08:00
jiangxingbo e635cbb6e6 [SPARK-18801][SQL][FOLLOWUP] Alias the view with its child
## What changes were proposed in this pull request?

This PR is a follow-up to address the comments https://github.com/apache/spark/pull/16233/files#r95669988 and https://github.com/apache/spark/pull/16233/files#r95662299.

We try to wrap the child by:
1. Generate the `queryOutput` by:
    1.1. If the query column names are defined, map the column names to attributes in the child output by name;
    1.2. Else set the child output attributes to `queryOutput`.
2. Map the `queryQutput` to view output by index, if the corresponding attributes don't match, try to up cast and alias the attribute in `queryOutput` to the attribute in the view output.
3. Add a Project over the child, with the new output generated by the previous steps.
If the view output doesn't have the same number of columns neither with the child output, nor with the query column names, throw an AnalysisException.

## How was this patch tested?

Add new test cases in `SQLViewSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #16561 from jiangxb1987/alias-view.
2017-01-16 19:11:21 +08:00
Wenchen Fan 6b34e745bb [SPARK-19178][SQL] convert string of large numbers to int should return null
## What changes were proposed in this pull request?

When we convert a string to integral, we will convert that string to `decimal(20, 0)` first, so that we can turn a string with decimal format to truncated integral, e.g. `CAST('1.2' AS int)` will return `1`.

However, this brings problems when we convert a string with large numbers to integral, e.g. `CAST('1234567890123' AS int)` will return `1912276171`, while Hive returns null as we expected.

This is a long standing bug(seems it was there the first day Spark SQL was created), this PR fixes this bug by adding the native support to convert `UTF8String` to integral.

## How was this patch tested?

new regression tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16550 from cloud-fan/string-to-int.
2017-01-12 22:52:34 -08:00
Takeshi YAMAMURO 5585ed93b0 [SPARK-17237][SQL] Remove backticks in a pivot result schema
## What changes were proposed in this pull request?
Pivoting adds backticks (e.g. 3_count(\`c\`)) in column names and, in some cases,
thes causes analysis exceptions  like;
```
scala> val df = Seq((2, 3, 4), (3, 4, 5)).toDF("a", "x", "y")
scala> df.groupBy("a").pivot("x").agg(count("y"), avg("y")).na.fill(0)
org.apache.spark.sql.AnalysisException: syntax error in attribute name: `3_count(`y`)`;
  at org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute$.e$1(unresolved.scala:134)
  at org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute$.parseAttributeName(unresolved.scala:144)
...
```
So, this pr proposes to remove these backticks from column names.

## How was this patch tested?
Added a test in `DataFrameAggregateSuite`.

Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes #14812 from maropu/SPARK-17237.
2017-01-12 09:46:53 -08:00
Wenchen Fan 871d266649 [SPARK-18969][SQL] Support grouping by nondeterministic expressions
## What changes were proposed in this pull request?

Currently nondeterministic expressions are allowed in `Aggregate`(see the [comment](https://github.com/apache/spark/blob/v2.0.2/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala#L249-L251)), but the `PullOutNondeterministic` analyzer rule failed to handle `Aggregate`, this PR fixes it.

close https://github.com/apache/spark/pull/16379

There is still one remaining issue: `SELECT a + rand() FROM t GROUP BY a + rand()` is not allowed, because the 2 `rand()` are different(we generate random seed as the default seed for `rand()`). https://issues.apache.org/jira/browse/SPARK-19035 is tracking this issue.

## How was this patch tested?

a new test suite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16404 from cloud-fan/groupby.
2017-01-12 20:21:04 +08:00
wangzhenhua 43fa21b3e6 [SPARK-19132][SQL] Add test cases for row size estimation and aggregate estimation
## What changes were proposed in this pull request?

In this pr, we add more test cases for project and aggregate estimation.

## How was this patch tested?

Add test cases.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #16551 from wzhfy/addTests.
2017-01-11 15:00:58 -08:00
Reynold Xin 66fe819ada [SPARK-19149][SQL] Follow-up: simplify cache implementation.
## What changes were proposed in this pull request?
This patch simplifies slightly the logical plan statistics cache implementation, as discussed in https://github.com/apache/spark/pull/16529

## How was this patch tested?
N/A - this has no behavior change.

Author: Reynold Xin <rxin@databricks.com>

Closes #16544 from rxin/SPARK-19149.
2017-01-11 14:25:36 -08:00
jiangxingbo 30a07071f0 [SPARK-18801][SQL] Support resolve a nested view
## What changes were proposed in this pull request?

We should be able to resolve a nested view. The main advantage is that if you update an underlying view, the current view also gets updated.
The new approach should be compatible with older versions of SPARK/HIVE, that means:
1. The new approach should be able to resolve the views that created by older versions of SPARK/HIVE;
2. The new approach should be able to resolve the views that are currently supported by SPARK SQL.

The new approach mainly brings in the following changes:
1. Add a new operator called `View` to keep track of the CatalogTable that describes the view, and the output attributes as well as the child of the view;
2. Update the `ResolveRelations` rule to resolve the relations and views, note that a nested view should be resolved correctly;
3. Add `viewDefaultDatabase` variable to `CatalogTable` to keep track of the default database name used to resolve a view, if the `CatalogTable` is not a view, then the variable should be `None`;
4. Add `AnalysisContext` to enable us to still support a view created with CTE/Windows query;
5. Enables the view support without enabling Hive support (i.e., enableHiveSupport);
6. Fix a weird behavior: the result of a view query may have different schema if the referenced table has been changed. After this PR, we try to cast the child output attributes to that from the view schema, throw an AnalysisException if cast is not allowed.

Note this is compatible with the views defined by older versions of Spark(before 2.2), which have empty `defaultDatabase` and all the relations in `viewText` have database part defined.

## How was this patch tested?
1. Add new tests in `SessionCatalogSuite` to test the function `lookupRelation`;
2. Add new test case in `SQLViewSuite` to test resolve a nested view.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #16233 from jiangxb1987/resolve-view.
2017-01-11 13:44:07 -08:00
wangzhenhua a615513569 [SPARK-19149][SQL] Unify two sets of statistics in LogicalPlan
## What changes were proposed in this pull request?

Currently we have two sets of statistics in LogicalPlan: a simple stats and a stats estimated by cbo, but the computing logic and naming are quite confusing, we need to unify these two sets of stats.

## How was this patch tested?

Just modify existing tests.

Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>

Closes #16529 from wzhfy/unifyStats.
2017-01-10 22:34:44 -08:00
Shixiong Zhu bc6c56e940 [SPARK-19140][SS] Allow update mode for non-aggregation streaming queries
## What changes were proposed in this pull request?

This PR allow update mode for non-aggregation streaming queries. It will be same as the append mode if a query has no aggregations.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16520 from zsxwing/update-without-agg.
2017-01-10 17:58:11 -08:00
Liwei Lin acfc5f3543 [SPARK-16845][SQL] GeneratedClass$SpecificOrdering grows beyond 64 KB
## What changes were proposed in this pull request?

Prior to this patch, we'll generate `compare(...)` for `GeneratedClass$SpecificOrdering` like below, leading to Janino exceptions saying the code grows beyond 64 KB.

``` scala
/* 005 */ class SpecificOrdering extends o.a.s.sql.catalyst.expressions.codegen.BaseOrdering {
/* ..... */   ...
/* 10969 */   private int compare(InternalRow a, InternalRow b) {
/* 10970 */     InternalRow i = null;  // Holds current row being evaluated.
/* 10971 */
/* 1.... */     code for comparing field0
/* 1.... */     code for comparing field1
/* 1.... */     ...
/* 1.... */     code for comparing field449
/* 15012 */
/* 15013 */     return 0;
/* 15014 */   }
/* 15015 */ }
```

This patch would break `compare(...)` into smaller `compare_xxx(...)` methods when necessary; then we'll get generated `compare(...)` like:

``` scala
/* 001 */ public SpecificOrdering generate(Object[] references) {
/* 002 */   return new SpecificOrdering(references);
/* 003 */ }
/* 004 */
/* 005 */ class SpecificOrdering extends o.a.s.sql.catalyst.expressions.codegen.BaseOrdering {
/* 006 */
/* 007 */     ...
/* 1.... */
/* 11290 */   private int compare_0(InternalRow a, InternalRow b) {
/* 11291 */     InternalRow i = null;  // Holds current row being evaluated.
/* 11292 */
/* 11293 */     i = a;
/* 11294 */     boolean isNullA;
/* 11295 */     UTF8String primitiveA;
/* 11296 */     {
/* 11297 */
/* 11298 */       Object obj = ((Expression) references[0]).eval(null);
/* 11299 */       UTF8String value = (UTF8String) obj;
/* 11300 */       isNullA = false;
/* 11301 */       primitiveA = value;
/* 11302 */     }
/* 11303 */     i = b;
/* 11304 */     boolean isNullB;
/* 11305 */     UTF8String primitiveB;
/* 11306 */     {
/* 11307 */
/* 11308 */       Object obj = ((Expression) references[0]).eval(null);
/* 11309 */       UTF8String value = (UTF8String) obj;
/* 11310 */       isNullB = false;
/* 11311 */       primitiveB = value;
/* 11312 */     }
/* 11313 */     if (isNullA && isNullB) {
/* 11314 */       // Nothing
/* 11315 */     } else if (isNullA) {
/* 11316 */       return -1;
/* 11317 */     } else if (isNullB) {
/* 11318 */       return 1;
/* 11319 */     } else {
/* 11320 */       int comp = primitiveA.compare(primitiveB);
/* 11321 */       if (comp != 0) {
/* 11322 */         return comp;
/* 11323 */       }
/* 11324 */     }
/* 11325 */
/* 11326 */
/* 11327 */     i = a;
/* 11328 */     boolean isNullA1;
/* 11329 */     UTF8String primitiveA1;
/* 11330 */     {
/* 11331 */
/* 11332 */       Object obj1 = ((Expression) references[1]).eval(null);
/* 11333 */       UTF8String value1 = (UTF8String) obj1;
/* 11334 */       isNullA1 = false;
/* 11335 */       primitiveA1 = value1;
/* 11336 */     }
/* 11337 */     i = b;
/* 11338 */     boolean isNullB1;
/* 11339 */     UTF8String primitiveB1;
/* 11340 */     {
/* 11341 */
/* 11342 */       Object obj1 = ((Expression) references[1]).eval(null);
/* 11343 */       UTF8String value1 = (UTF8String) obj1;
/* 11344 */       isNullB1 = false;
/* 11345 */       primitiveB1 = value1;
/* 11346 */     }
/* 11347 */     if (isNullA1 && isNullB1) {
/* 11348 */       // Nothing
/* 11349 */     } else if (isNullA1) {
/* 11350 */       return -1;
/* 11351 */     } else if (isNullB1) {
/* 11352 */       return 1;
/* 11353 */     } else {
/* 11354 */       int comp = primitiveA1.compare(primitiveB1);
/* 11355 */       if (comp != 0) {
/* 11356 */         return comp;
/* 11357 */       }
/* 11358 */     }
/* 1.... */
/* 1.... */   ...
/* 1.... */
/* 12652 */     return 0;
/* 12653 */   }
/* 1.... */
/* 1.... */   ...
/* 15387 */
/* 15388 */   public int compare(InternalRow a, InternalRow b) {
/* 15389 */
/* 15390 */     int comp_0 = compare_0(a, b);
/* 15391 */     if (comp_0 != 0) {
/* 15392 */       return comp_0;
/* 15393 */     }
/* 15394 */
/* 15395 */     int comp_1 = compare_1(a, b);
/* 15396 */     if (comp_1 != 0) {
/* 15397 */       return comp_1;
/* 15398 */     }
/* 1.... */
/* 1.... */     ...
/* 1.... */
/* 15450 */     return 0;
/* 15451 */   }
/* 15452 */ }
```
## How was this patch tested?
- a new added test case which
  - would fail prior to this patch
  - would pass with this patch
- ordering correctness should already be covered by existing tests like those in `OrderingSuite`

## Acknowledgement

A major part of this PR - the refactoring work of `splitExpression()` - has been done by ueshin.

Author: Liwei Lin <lwlin7@gmail.com>
Author: Takuya UESHIN <ueshin@happy-camper.st>
Author: Takuya Ueshin <ueshin@happy-camper.st>

Closes #15480 from lw-lin/spec-ordering-64k-.
2017-01-10 19:35:46 +08:00
Zhenhua Wang 15c2bd01b0 [SPARK-19020][SQL] Cardinality estimation of aggregate operator
## What changes were proposed in this pull request?

Support cardinality estimation of aggregate operator

## How was this patch tested?

Add test cases

Author: Zhenhua Wang <wzh_zju@163.com>
Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #16431 from wzhfy/aggEstimation.
2017-01-09 11:29:42 -08:00
Zhenhua Wang 3ccabdfb4d [SPARK-17077][SQL] Cardinality estimation for project operator
## What changes were proposed in this pull request?

Support cardinality estimation for project operator.

## How was this patch tested?

Add a test suite and a base class in the catalyst package.

Author: Zhenhua Wang <wzh_zju@163.com>

Closes #16430 from wzhfy/projectEstimation.
2017-01-08 21:15:52 -08:00
Michal Senkyr 903bb8e8a2 [SPARK-16792][SQL] Dataset containing a Case Class with a List type causes a CompileException (converting sequence to list)
## What changes were proposed in this pull request?

Added a `to` call at the end of the code generated by `ScalaReflection.deserializerFor` if the requested type is not a supertype of `WrappedArray[_]` that uses `CanBuildFrom[_, _, _]` to convert result into an arbitrary subtype of `Seq[_]`.

Care was taken to preserve the original deserialization where it is possible to avoid the overhead of conversion in cases where it is not needed

`ScalaReflection.serializerFor` could already be used to serialize any `Seq[_]` so it was not altered

`SQLImplicits` had to be altered and new implicit encoders added to permit serialization of other sequence types

Also fixes [SPARK-16815] Dataset[List[T]] leads to ArrayStoreException

## How was this patch tested?
```bash
./build/mvn -DskipTests clean package && ./dev/run-tests
```

Also manual execution of the following sets of commands in the Spark shell:
```scala
case class TestCC(key: Int, letters: List[String])

val ds1 = sc.makeRDD(Seq(
(List("D")),
(List("S","H")),
(List("F","H")),
(List("D","L","L"))
)).map(x=>(x.length,x)).toDF("key","letters").as[TestCC]

val test1=ds1.map{_.key}
test1.show
```

```scala
case class X(l: List[String])
spark.createDataset(Seq(List("A"))).map(X).show
```

```scala
spark.sqlContext.createDataset(sc.parallelize(List(1) :: Nil)).collect
```

After adding arbitrary sequence support also tested with the following commands:

```scala
case class QueueClass(q: scala.collection.immutable.Queue[Int])

spark.createDataset(Seq(List(1,2,3))).map(x => QueueClass(scala.collection.immutable.Queue(x: _*))).map(_.q.dequeue).collect
```

Author: Michal Senkyr <mike.senkyr@gmail.com>

Closes #16240 from michalsenkyr/sql-caseclass-list-fix.
2017-01-06 15:05:20 +08:00
Wenchen Fan cca945b6aa [SPARK-18885][SQL] unify CREATE TABLE syntax for data source and hive serde tables
## What changes were proposed in this pull request?

Today we have different syntax to create data source or hive serde tables, we should unify them to not confuse users and step forward to make hive a data source.

Please read https://issues.apache.org/jira/secure/attachment/12843835/CREATE-TABLE.pdf for  details.

TODO(for follow-up PRs):
1. TBLPROPERTIES is not added to the new syntax, we should decide if we wanna add it later.
2. `SHOW CREATE TABLE` should be updated to use the new syntax.
3. we should decide if we wanna change the behavior of `SET LOCATION`.

## How was this patch tested?

new tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16296 from cloud-fan/create-table.
2017-01-05 17:40:27 -08:00
Niranjan Padmanabhan a1e40b1f5d
[MINOR][DOCS] Remove consecutive duplicated words/typo in Spark Repo
## What changes were proposed in this pull request?
There are many locations in the Spark repo where the same word occurs consecutively. Sometimes they are appropriately placed, but many times they are not. This PR removes the inappropriately duplicated words.

## How was this patch tested?
N/A since only docs or comments were updated.

Author: Niranjan Padmanabhan <niranjan.padmanabhan@gmail.com>

Closes #16455 from neurons/np.structure_streaming_doc.
2017-01-04 15:07:29 +00:00
Wenchen Fan 101556d0fa [SPARK-19060][SQL] remove the supportsPartial flag in AggregateFunction
## What changes were proposed in this pull request?

Now all aggregation functions support partial aggregate, we can remove the `supportsPartual` flag in `AggregateFunction`

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16461 from cloud-fan/partial.
2017-01-04 12:46:30 +01:00
Wenchen Fan cbd11d2357 [SPARK-19072][SQL] codegen of Literal should not output boxed value
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/16402 we made a mistake that, when double/float is infinity, the `Literal` codegen will output boxed value and cause wrong result.

This PR fixes this by special handling infinity to not output boxed value.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16469 from cloud-fan/literal.
2017-01-03 22:40:14 -08:00
gatorsmile b67b35f76b [SPARK-19048][SQL] Delete Partition Location when Dropping Managed Partitioned Tables in InMemoryCatalog
### What changes were proposed in this pull request?
The data in the managed table should be deleted after table is dropped. However, if the partition location is not under the location of the partitioned table, it is not deleted as expected. Users can specify any location for the partition when they adding a partition.

This PR is to delete partition location when dropping managed partitioned tables stored in `InMemoryCatalog`.

### How was this patch tested?
Added test cases for both HiveExternalCatalog and InMemoryCatalog

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16448 from gatorsmile/unsetSerdeProp.
2017-01-03 11:43:47 -08:00
Liang-Chi Hsieh 52636226dc [SPARK-18932][SQL] Support partial aggregation for collect_set/collect_list
## What changes were proposed in this pull request?

Currently collect_set/collect_list aggregation expression don't support partial aggregation. This patch is to enable partial aggregation for them.

## How was this patch tested?

Jenkins tests.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #16371 from viirya/collect-partial-support.
2017-01-03 22:11:54 +08:00
Zhenhua Wang ae83c21125 [SPARK-18998][SQL] Add a cbo conf to switch between default statistics and estimated statistics
## What changes were proposed in this pull request?

We add a cbo configuration to switch between default stats and estimated stats.
We also define a new statistics method `planStats` in LogicalPlan with conf as its parameter, in order to pass the cbo switch and other estimation related configurations in the future. `planStats` is used on the caller sides (i.e. in Optimizer and Strategies) to make transformation decisions based on stats.

## How was this patch tested?

Add a test case using a dummy LogicalPlan.

Author: Zhenhua Wang <wzh_zju@163.com>

Closes #16401 from wzhfy/cboSwitch.
2017-01-03 12:19:52 +08:00
gatorsmile a6cd9dbc60 [SPARK-19029][SQL] Remove databaseName from SimpleCatalogRelation
### What changes were proposed in this pull request?
Remove useless `databaseName ` from `SimpleCatalogRelation`.

### How was this patch tested?
Existing test cases.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16438 from gatorsmile/removeDBFromSimpleCatalogRelation.
2017-01-03 11:55:31 +08:00
gatorsmile 35e974076d [SPARK-19028][SQL] Fixed non-thread-safe functions used in SessionCatalog
### What changes were proposed in this pull request?
Fixed non-thread-safe functions used in SessionCatalog:
- refreshTable
- lookupRelation

### How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16437 from gatorsmile/addSyncToLookUpTable.
2016-12-31 19:40:28 +08:00
hyukjinkwon 852782b83c
[SPARK-18922][TESTS] Fix more path-related test failures on Windows
## What changes were proposed in this pull request?

This PR proposes to fix the test failures due to different format of paths on Windows.

Failed tests are as below:

```
ColumnExpressionSuite:
- input_file_name, input_file_block_start, input_file_block_length - FileScanRDD *** FAILED *** (187 milliseconds)
  "file:///C:/projects/spark/target/tmp/spark-0b21b963-6cfa-411c-8d6f-e6a5e1e73bce/part-00001-c083a03a-e55e-4b05-9073-451de352d006.snappy.parquet" did not contain "C:\projects\spark\target\tmp\spark-0b21b963-6cfa-411c-8d6f-e6a5e1e73bce" (ColumnExpressionSuite.scala:545)

- input_file_name, input_file_block_start, input_file_block_length - HadoopRDD *** FAILED *** (172 milliseconds)
  "file:/C:/projects/spark/target/tmp/spark-5d0afa94-7c2f-463b-9db9-2e8403e2bc5f/part-00000-f6530138-9ad3-466d-ab46-0eeb6f85ed0b.txt" did not contain "C:\projects\spark\target\tmp\spark-5d0afa94-7c2f-463b-9db9-2e8403e2bc5f" (ColumnExpressionSuite.scala:569)

- input_file_name, input_file_block_start, input_file_block_length - NewHadoopRDD *** FAILED *** (156 milliseconds)
  "file:/C:/projects/spark/target/tmp/spark-a894c7df-c74d-4d19-82a2-a04744cb3766/part-00000-29674e3f-3fcf-4327-9b04-4dab1d46338d.txt" did not contain "C:\projects\spark\target\tmp\spark-a894c7df-c74d-4d19-82a2-a04744cb3766" (ColumnExpressionSuite.scala:598)
```

```
DataStreamReaderWriterSuite:
- source metadataPath *** FAILED *** (62 milliseconds)
  org.mockito.exceptions.verification.junit.ArgumentsAreDifferent: Argument(s) are different! Wanted:
streamSourceProvider.createSource(
    org.apache.spark.sql.SQLContext3b04133b,
    "C:\projects\spark\target\tmp\streaming.metadata-b05db6ae-c8dc-4ce4-b0d9-1eb8c84876c0/sources/0",
    None,
    "org.apache.spark.sql.streaming.test",
    Map()
);
-> at org.apache.spark.sql.streaming.test.DataStreamReaderWriterSuite$$anonfun$12.apply$mcV$sp(DataStreamReaderWriterSuite.scala:374)
Actual invocation has different arguments:
streamSourceProvider.createSource(
    org.apache.spark.sql.SQLContext3b04133b,
    "/C:/projects/spark/target/tmp/streaming.metadata-b05db6ae-c8dc-4ce4-b0d9-1eb8c84876c0/sources/0",
    None,
    "org.apache.spark.sql.streaming.test",
    Map()
);
```

```
GlobalTempViewSuite:
- CREATE GLOBAL TEMP VIEW USING *** FAILED *** (110 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark  arget mpspark-960398ba-a0a1-45f6-a59a-d98533f9f519;
```

```
CreateTableAsSelectSuite:
- CREATE TABLE USING AS SELECT *** FAILED *** (0 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string

- create a table, drop it and create another one with the same name *** FAILED *** (16 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string

- create table using as select - with partitioned by *** FAILED *** (0 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string

- create table using as select - with non-zero buckets *** FAILED *** (0 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string
```

```
HiveMetadataCacheSuite:
- partitioned table is cached when partition pruning is true *** FAILED *** (532 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- partitioned table is cached when partition pruning is false *** FAILED *** (297 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
MultiDatabaseSuite:
- createExternalTable() to non-default database - with USE *** FAILED *** (954 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark  arget mpspark-0839d9a7-5e29-467a-9e3e-3e4cd618ee09;

- createExternalTable() to non-default database - without USE *** FAILED *** (500 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark  arget mpspark-c7e24d73-1d8f-45e8-ab7d-53a83087aec3;

 - invalid database name and table names *** FAILED *** (31 milliseconds)
   "Path does not exist: file:/C:projectsspark  arget mpspark-15a2a494-3483-4876-80e5-ec396e704b77;" did not contain "`t:a` is not a valid name for tables/databases. Valid names only contain alphabet characters, numbers and _." (MultiDatabaseSuite.scala:296)
```

```
OrcQuerySuite:
 - SPARK-8501: Avoids discovery schema from empty ORC files *** FAILED *** (15 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - Verify the ORC conversion parameter: CONVERT_METASTORE_ORC *** FAILED *** (78 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - converted ORC table supports resolving mixed case field *** FAILED *** (297 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
HadoopFsRelationTest - JsonHadoopFsRelationSuite, OrcHadoopFsRelationSuite, ParquetHadoopFsRelationSuite, SimpleTextHadoopFsRelationSuite:
 - Locality support for FileScanRDD *** FAILED *** (15 milliseconds)
   java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-383d1f13-8783-47fd-964d-9c75e5eec50f, expected: file:///
```

```
HiveQuerySuite:
- CREATE TEMPORARY FUNCTION *** FAILED *** (0 milliseconds)
   java.net.MalformedURLException: For input string: "%5Cprojects%5Cspark%5Csql%5Chive%5Ctarget%5Cscala-2.11%5Ctest-classes%5CTestUDTF.jar"

 - ADD FILE command *** FAILED *** (500 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\sql\hive\target\scala-2.11\test-classes\data\files\v1.txt

 - ADD JAR command 2 *** FAILED *** (110 milliseconds)
   org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive  argetscala-2.11 est-classesdatafilessample.json;
```

```
PruneFileSourcePartitionsSuite:
 - PruneFileSourcePartitions should not change the output of LogicalRelation *** FAILED *** (15 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
HiveCommandSuite:
 - LOAD DATA LOCAL *** FAILED *** (109 milliseconds)
   org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive  argetscala-2.11 est-classesdatafilesemployee.dat;

 - LOAD DATA *** FAILED *** (93 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 15: C:projectsspark arget mpemployee.dat7496657117354281006.tmp

 - Truncate Table *** FAILED *** (78 milliseconds)
   org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive  argetscala-2.11 est-classesdatafilesemployee.dat;
```

```
HiveExternalCatalogBackwardCompatibilitySuite:
- make sure we can read table created by old version of Spark *** FAILED *** (0 milliseconds)
  "[/C:/projects/spark/target/tmp/]spark-0554d859-74e1-..." did not equal "[C:\projects\spark\target\tmp\]spark-0554d859-74e1-..." (HiveExternalCatalogBackwardCompatibilitySuite.scala:213)
  org.scalatest.exceptions.TestFailedException

- make sure we can alter table location created by old version of Spark *** FAILED *** (110 milliseconds)
  java.net.URISyntaxException: Illegal character in opaque part at index 15: C:projectsspark	arget	mpspark-0e9b2c5f-49a1-4e38-a32a-c0ab1813a79f
```

```
ExternalCatalogSuite:
- create/drop/rename partitions should create/delete/rename the directory *** FAILED *** (610 milliseconds)
  java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-4c24f010-18df-437b-9fed-990c6f9adece
```

```
SQLQuerySuite:
- describe functions - temporary user defined functions *** FAILED *** (16 milliseconds)
  java.net.URISyntaxException: Illegal character in opaque part at index 22: C:projectssparksqlhive	argetscala-2.11	est-classesTestUDTF.jar

- specifying database name for a temporary table is not allowed *** FAILED *** (125 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-a34c9814-a483-43f2-be29-37f616b6df91;
```

```
PartitionProviderCompatibilitySuite:
- convert partition provider to hive with repair table *** FAILED *** (281 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-ee5fc96d-8c7d-4ebf-8571-a1d62736473e;

- when partition management is enabled, new tables have partition provider hive *** FAILED *** (187 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-803ad4d6-3e8c-498d-9ca5-5cda5d9b2a48;

- when partition management is disabled, new tables have no partition provider *** FAILED *** (172 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-c9fda9e2-4020-465f-8678-52cd72d0a58f;

- when partition management is disabled, we preserve the old behavior even for new tables *** FAILED *** (203 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget
mpspark-f4a518a6-c49d-43d3-b407-0ddd76948e13;

- insert overwrite partition of legacy datasource table *** FAILED *** (188 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-f4a518a6-c49d-43d3-b407-0ddd76948e79;

- insert overwrite partition of new datasource table overwrites just partition *** FAILED *** (219 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-6ba3a88d-6f6c-42c5-a9f4-6d924a0616ff;

- SPARK-18544 append with saveAsTable - partition management true *** FAILED *** (173 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-cd234a6d-9cb4-4d1d-9e51-854ae9543bbd;

- SPARK-18635 special chars in partition values - partition management true *** FAILED *** (2 seconds, 967 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- SPARK-18635 special chars in partition values - partition management false *** FAILED *** (62 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- SPARK-18659 insert overwrite table with lowercase - partition management true *** FAILED *** (63 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- SPARK-18544 append with saveAsTable - partition management false *** FAILED *** (266 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- SPARK-18659 insert overwrite table files - partition management false *** FAILED *** (63 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- SPARK-18659 insert overwrite table with lowercase - partition management false *** FAILED *** (78 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- sanity check table setup *** FAILED *** (31 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- insert into partial dynamic partitions *** FAILED *** (47 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- insert into fully dynamic partitions *** FAILED *** (62 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- insert into static partition *** FAILED *** (78 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- overwrite partial dynamic partitions *** FAILED *** (63 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- overwrite fully dynamic partitions *** FAILED *** (47 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- overwrite static partition *** FAILED *** (63 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
MetastoreDataSourcesSuite:
- check change without refresh *** FAILED *** (203 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-00713fe4-ca04-448c-bfc7-6c5e9a2ad2a1;

- drop, change, recreate *** FAILED *** (78 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-2030a21b-7d67-4385-a65b-bb5e2bed4861;

- SPARK-15269 external data source table creation *** FAILED *** (78 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-4d50fd4a-14bc-41d6-9232-9554dd233f86;

- CTAS *** FAILED *** (109 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string

- CTAS with IF NOT EXISTS *** FAILED *** (109 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string

- CTAS: persisted partitioned bucketed data source table *** FAILED *** (0 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string

- SPARK-15025: create datasource table with path with select *** FAILED *** (16 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string

- CTAS: persisted partitioned data source table *** FAILED *** (47 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string
```

```
HiveMetastoreCatalogSuite:
- Persist non-partitioned parquet relation into metastore as managed table using CTAS *** FAILED *** (16 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string

- Persist non-partitioned orc relation into metastore as managed table using CTAS *** FAILED *** (16 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string
```

```
HiveUDFSuite:
- SPARK-11522 select input_file_name from non-parquet table *** FAILED *** (16 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
QueryPartitionSuite:
- SPARK-13709: reading partitioned Avro table with nested schema *** FAILED *** (250 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
ParquetHiveCompatibilitySuite:
- simple primitives *** FAILED *** (16 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- SPARK-10177 timestamp *** FAILED *** (0 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- array *** FAILED *** (16 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- map *** FAILED *** (16 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- struct *** FAILED *** (0 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- SPARK-16344: array of struct with a single field named 'array_element' *** FAILED *** (15 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

## How was this patch tested?

Manually tested via AppVeyor.

```
ColumnExpressionSuite:
- input_file_name, input_file_block_start, input_file_block_length - FileScanRDD (234 milliseconds)
- input_file_name, input_file_block_start, input_file_block_length - HadoopRDD (235 milliseconds)
- input_file_name, input_file_block_start, input_file_block_length - NewHadoopRDD (203 milliseconds)
```

```
DataStreamReaderWriterSuite:
- source metadataPath (63 milliseconds)
```

```
GlobalTempViewSuite:
 - CREATE GLOBAL TEMP VIEW USING (436 milliseconds)
```

```
CreateTableAsSelectSuite:
- CREATE TABLE USING AS SELECT (171 milliseconds)
- create a table, drop it and create another one with the same name (422 milliseconds)
- create table using as select - with partitioned by (141 milliseconds)
- create table using as select - with non-zero buckets (125 milliseconds)
```

```
HiveMetadataCacheSuite:
- partitioned table is cached when partition pruning is true (3 seconds, 211 milliseconds)
- partitioned table is cached when partition pruning is false (1 second, 781 milliseconds)
```

```
MultiDatabaseSuite:
 - createExternalTable() to non-default database - with USE (797 milliseconds)
 - createExternalTable() to non-default database - without USE (640 milliseconds)
 - invalid database name and table names (62 milliseconds)
```

```
OrcQuerySuite:
 - SPARK-8501: Avoids discovery schema from empty ORC files (703 milliseconds)
 - Verify the ORC conversion parameter: CONVERT_METASTORE_ORC (750 milliseconds)
 - converted ORC table supports resolving mixed case field (625 milliseconds)
```

```
HadoopFsRelationTest - JsonHadoopFsRelationSuite, OrcHadoopFsRelationSuite, ParquetHadoopFsRelationSuite, SimpleTextHadoopFsRelationSuite:
 - Locality support for FileScanRDD (296 milliseconds)
```

```
HiveQuerySuite:
 - CREATE TEMPORARY FUNCTION (125 milliseconds)
 - ADD FILE command (250 milliseconds)
 - ADD JAR command 2 (609 milliseconds)
```

```
PruneFileSourcePartitionsSuite:
- PruneFileSourcePartitions should not change the output of LogicalRelation (359 milliseconds)
```

```
HiveCommandSuite:
 - LOAD DATA LOCAL (1 second, 829 milliseconds)
 - LOAD DATA (1 second, 735 milliseconds)
 - Truncate Table (1 second, 641 milliseconds)
```

```
HiveExternalCatalogBackwardCompatibilitySuite:
 - make sure we can read table created by old version of Spark (32 milliseconds)
 - make sure we can alter table location created by old version of Spark (125 milliseconds)
 - make sure we can rename table created by old version of Spark (281 milliseconds)
```

```
ExternalCatalogSuite:
- create/drop/rename partitions should create/delete/rename the directory (625 milliseconds)
```

```
SQLQuerySuite:
- describe functions - temporary user defined functions (31 milliseconds)
- specifying database name for a temporary table is not allowed (390 milliseconds)
```

```
PartitionProviderCompatibilitySuite:
 - convert partition provider to hive with repair table (813 milliseconds)
 - when partition management is enabled, new tables have partition provider hive (562 milliseconds)
 - when partition management is disabled, new tables have no partition provider (344 milliseconds)
 - when partition management is disabled, we preserve the old behavior even for new tables (422 milliseconds)
 - insert overwrite partition of legacy datasource table (750 milliseconds)
 - SPARK-18544 append with saveAsTable - partition management true (985 milliseconds)
 - SPARK-18635 special chars in partition values - partition management true (3 seconds, 328 milliseconds)
 - SPARK-18635 special chars in partition values - partition management false (2 seconds, 891 milliseconds)
 - SPARK-18659 insert overwrite table with lowercase - partition management true (750 milliseconds)
 - SPARK-18544 append with saveAsTable - partition management false (656 milliseconds)
 - SPARK-18659 insert overwrite table files - partition management false (922 milliseconds)
 - SPARK-18659 insert overwrite table with lowercase - partition management false (469 milliseconds)
 - sanity check table setup (937 milliseconds)
 - insert into partial dynamic partitions (2 seconds, 985 milliseconds)
 - insert into fully dynamic partitions (1 second, 937 milliseconds)
 - insert into static partition (1 second, 578 milliseconds)
 - overwrite partial dynamic partitions (7 seconds, 561 milliseconds)
 - overwrite fully dynamic partitions (1 second, 766 milliseconds)
 - overwrite static partition (1 second, 797 milliseconds)
```

```
MetastoreDataSourcesSuite:
 - check change without refresh (610 milliseconds)
 - drop, change, recreate (437 milliseconds)
 - SPARK-15269 external data source table creation (297 milliseconds)
 - CTAS with IF NOT EXISTS (437 milliseconds)
 - CTAS: persisted partitioned bucketed data source table (422 milliseconds)
 - SPARK-15025: create datasource table with path with select (265 milliseconds)
 - CTAS (438 milliseconds)
 - CTAS with IF NOT EXISTS (469 milliseconds)
 - CTAS: persisted partitioned bucketed data source table (406 milliseconds)
```

```
HiveMetastoreCatalogSuite:
 - Persist non-partitioned parquet relation into metastore as managed table using CTAS (406 milliseconds)
 - Persist non-partitioned orc relation into metastore as managed table using CTAS (313 milliseconds)
```

```
HiveUDFSuite:
 - SPARK-11522 select input_file_name from non-parquet table (3 seconds, 144 milliseconds)
```

```
QueryPartitionSuite:
 - SPARK-13709: reading partitioned Avro table with nested schema (1 second, 67 milliseconds)
```

```
ParquetHiveCompatibilitySuite:
 - simple primitives (745 milliseconds)
 - SPARK-10177 timestamp (375 milliseconds)
 - array (407 milliseconds)
 - map (409 milliseconds)
 - struct (437 milliseconds)
 - SPARK-16344: array of struct with a single field named 'array_element' (391 milliseconds)
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16397 from HyukjinKwon/SPARK-18922-paths.
2016-12-30 11:16:03 +00:00
Kazuaki Ishizaki 93f35569fd [SPARK-16213][SQL] Reduce runtime overhead of a program that creates an primitive array in DataFrame
## What changes were proposed in this pull request?

This PR reduces runtime overhead of a program the creates an primitive array in DataFrame by using the similar approach to #15044. Generated code performs boxing operation in an assignment from InternalRow to an `Object[]` temporary array (at Lines 051 and 061 in the generated code before without this PR). If we know that type of array elements is primitive, we apply the following optimizations:
1. Eliminate a pair of `isNullAt()` and a null assignment
2. Allocate an primitive array instead of `Object[]` (eliminate boxing operations)
3. Create `UnsafeArrayData` by using `UnsafeArrayWriter` to keep a primitive array in a row format instead of doing non-lightweight operations in constructor of `GenericArrayData`
The PR also performs the same things for `CreateMap`.

Here are performance results of [DataFrame programs](6bf54ec5e2/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala (L83-L112)) by up to 17.9x over without this PR.

```
Without SPARK-16043
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)
Read a primitive array in DataFrame:     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                           3805 / 4150          0.0      507308.9       1.0X
Double                                        3593 / 3852          0.0      479056.9       1.1X

With SPARK-16043
Read a primitive array in DataFrame:     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            213 /  271          0.0       28387.5       1.0X
Double                                         204 /  223          0.0       27250.9       1.0X
```
Note : #15780 is enabled for these measurements

An motivating example

``` java
val df = sparkContext.parallelize(Seq(0.0d, 1.0d), 1).toDF
df.selectExpr("Array(value + 1.1d, value + 2.2d)").show
```

Generated code without this PR

``` java
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private scala.collection.Iterator inputadapter_input;
/* 009 */   private UnsafeRow serializefromobject_result;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 012 */   private Object[] project_values;
/* 013 */   private UnsafeRow project_result;
/* 014 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder project_holder;
/* 015 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter project_rowWriter;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter project_arrayWriter;
/* 017 */
/* 018 */   public GeneratedIterator(Object[] references) {
/* 019 */     this.references = references;
/* 020 */   }
/* 021 */
/* 022 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 023 */     partitionIndex = index;
/* 024 */     this.inputs = inputs;
/* 025 */     inputadapter_input = inputs[0];
/* 026 */     serializefromobject_result = new UnsafeRow(1);
/* 027 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 0);
/* 028 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 029 */     this.project_values = null;
/* 030 */     project_result = new UnsafeRow(1);
/* 031 */     this.project_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(project_result, 32);
/* 032 */     this.project_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(project_holder, 1);
/* 033 */     this.project_arrayWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
/* 034 */
/* 035 */   }
/* 036 */
/* 037 */   protected void processNext() throws java.io.IOException {
/* 038 */     while (inputadapter_input.hasNext()) {
/* 039 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 040 */       double inputadapter_value = inputadapter_row.getDouble(0);
/* 041 */
/* 042 */       final boolean project_isNull = false;
/* 043 */       this.project_values = new Object[2];
/* 044 */       boolean project_isNull1 = false;
/* 045 */
/* 046 */       double project_value1 = -1.0;
/* 047 */       project_value1 = inputadapter_value + 1.1D;
/* 048 */       if (false) {
/* 049 */         project_values[0] = null;
/* 050 */       } else {
/* 051 */         project_values[0] = project_value1;
/* 052 */       }
/* 053 */
/* 054 */       boolean project_isNull4 = false;
/* 055 */
/* 056 */       double project_value4 = -1.0;
/* 057 */       project_value4 = inputadapter_value + 2.2D;
/* 058 */       if (false) {
/* 059 */         project_values[1] = null;
/* 060 */       } else {
/* 061 */         project_values[1] = project_value4;
/* 062 */       }
/* 063 */
/* 064 */       final ArrayData project_value = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_values);
/* 065 */       this.project_values = null;
/* 066 */       project_holder.reset();
/* 067 */
/* 068 */       project_rowWriter.zeroOutNullBytes();
/* 069 */
/* 070 */       if (project_isNull) {
/* 071 */         project_rowWriter.setNullAt(0);
/* 072 */       } else {
/* 073 */         // Remember the current cursor so that we can calculate how many bytes are
/* 074 */         // written later.
/* 075 */         final int project_tmpCursor = project_holder.cursor;
/* 076 */
/* 077 */         if (project_value instanceof UnsafeArrayData) {
/* 078 */           final int project_sizeInBytes = ((UnsafeArrayData) project_value).getSizeInBytes();
/* 079 */           // grow the global buffer before writing data.
/* 080 */           project_holder.grow(project_sizeInBytes);
/* 081 */           ((UnsafeArrayData) project_value).writeToMemory(project_holder.buffer, project_holder.cursor);
/* 082 */           project_holder.cursor += project_sizeInBytes;
/* 083 */
/* 084 */         } else {
/* 085 */           final int project_numElements = project_value.numElements();
/* 086 */           project_arrayWriter.initialize(project_holder, project_numElements, 8);
/* 087 */
/* 088 */           for (int project_index = 0; project_index < project_numElements; project_index++) {
/* 089 */             if (project_value.isNullAt(project_index)) {
/* 090 */               project_arrayWriter.setNullDouble(project_index);
/* 091 */             } else {
/* 092 */               final double project_element = project_value.getDouble(project_index);
/* 093 */               project_arrayWriter.write(project_index, project_element);
/* 094 */             }
/* 095 */           }
/* 096 */         }
/* 097 */
/* 098 */         project_rowWriter.setOffsetAndSize(0, project_tmpCursor, project_holder.cursor - project_tmpCursor);
/* 099 */       }
/* 100 */       project_result.setTotalSize(project_holder.totalSize());
/* 101 */       append(project_result);
/* 102 */       if (shouldStop()) return;
/* 103 */     }
/* 104 */   }
/* 105 */ }
```

Generated code with this PR

``` java
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private scala.collection.Iterator inputadapter_input;
/* 009 */   private UnsafeRow serializefromobject_result;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 012 */   private UnsafeArrayData project_arrayData;
/* 013 */   private UnsafeRow project_result;
/* 014 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder project_holder;
/* 015 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter project_rowWriter;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter project_arrayWriter;
/* 017 */
/* 018 */   public GeneratedIterator(Object[] references) {
/* 019 */     this.references = references;
/* 020 */   }
/* 021 */
/* 022 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 023 */     partitionIndex = index;
/* 024 */     this.inputs = inputs;
/* 025 */     inputadapter_input = inputs[0];
/* 026 */     serializefromobject_result = new UnsafeRow(1);
/* 027 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 0);
/* 028 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 029 */
/* 030 */     project_result = new UnsafeRow(1);
/* 031 */     this.project_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(project_result, 32);
/* 032 */     this.project_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(project_holder, 1);
/* 033 */     this.project_arrayWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
/* 034 */
/* 035 */   }
/* 036 */
/* 037 */   protected void processNext() throws java.io.IOException {
/* 038 */     while (inputadapter_input.hasNext()) {
/* 039 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 040 */       double inputadapter_value = inputadapter_row.getDouble(0);
/* 041 */
/* 042 */       byte[] project_array = new byte[32];
/* 043 */       project_arrayData = new UnsafeArrayData();
/* 044 */       Platform.putLong(project_array, 16, 2);
/* 045 */       project_arrayData.pointTo(project_array, 16, 32);
/* 046 */
/* 047 */       boolean project_isNull1 = false;
/* 048 */
/* 049 */       double project_value1 = -1.0;
/* 050 */       project_value1 = inputadapter_value + 1.1D;
/* 051 */       if (false) {
/* 052 */         project_arrayData.setNullAt(0);
/* 053 */       } else {
/* 054 */         project_arrayData.setDouble(0, project_value1);
/* 055 */       }
/* 056 */
/* 057 */       boolean project_isNull4 = false;
/* 058 */
/* 059 */       double project_value4 = -1.0;
/* 060 */       project_value4 = inputadapter_value + 2.2D;
/* 061 */       if (false) {
/* 062 */         project_arrayData.setNullAt(1);
/* 063 */       } else {
/* 064 */         project_arrayData.setDouble(1, project_value4);
/* 065 */       }
/* 066 */       project_holder.reset();
/* 067 */
/* 068 */       // Remember the current cursor so that we can calculate how many bytes are
/* 069 */       // written later.
/* 070 */       final int project_tmpCursor = project_holder.cursor;
/* 071 */
/* 072 */       if (project_arrayData instanceof UnsafeArrayData) {
/* 073 */         final int project_sizeInBytes = ((UnsafeArrayData) project_arrayData).getSizeInBytes();
/* 074 */         // grow the global buffer before writing data.
/* 075 */         project_holder.grow(project_sizeInBytes);
/* 076 */         ((UnsafeArrayData) project_arrayData).writeToMemory(project_holder.buffer, project_holder.cursor);
/* 077 */         project_holder.cursor += project_sizeInBytes;
/* 078 */
/* 079 */       } else {
/* 080 */         final int project_numElements = project_arrayData.numElements();
/* 081 */         project_arrayWriter.initialize(project_holder, project_numElements, 8);
/* 082 */
/* 083 */         for (int project_index = 0; project_index < project_numElements; project_index++) {
/* 084 */           if (project_arrayData.isNullAt(project_index)) {
/* 085 */             project_arrayWriter.setNullDouble(project_index);
/* 086 */           } else {
/* 087 */             final double project_element = project_arrayData.getDouble(project_index);
/* 088 */             project_arrayWriter.write(project_index, project_element);
/* 089 */           }
/* 090 */         }
/* 091 */       }
/* 092 */
/* 093 */       project_rowWriter.setOffsetAndSize(0, project_tmpCursor, project_holder.cursor - project_tmpCursor);
/* 094 */       project_result.setTotalSize(project_holder.totalSize());
/* 095 */       append(project_result);
/* 096 */       if (shouldStop()) return;
/* 097 */     }
/* 098 */   }
/* 099 */ }
```
## How was this patch tested?

Added unit tests into `DataFrameComplexTypeSuite`

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #13909 from kiszk/SPARK-16213.
2016-12-29 10:59:37 +08:00
Wenchen Fan 6ddbf467b4 [SPARK-18999][SQL][MINOR] simplify Literal codegen
## What changes were proposed in this pull request?

`Literal` can use `CodegenContex.addReferenceObj` to implement codegen, instead of `CodegenFallback`.  This can also simplify the generated code a little bit, before we will generate: `((Expression) references[1]).eval(null)`, now it's just `references[1]`.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16402 from cloud-fan/minor.
2016-12-27 06:22:12 -08:00
Wenchen Fan 8a7db8a608 [SPARK-18980][SQL] implement Aggregator with TypedImperativeAggregate
## What changes were proposed in this pull request?

Currently we implement `Aggregator` with `DeclarativeAggregate`, which will serialize/deserialize the buffer object every time we process an input.

This PR implements `Aggregator` with `TypedImperativeAggregate` and avoids to serialize/deserialize buffer object many times. The benchmark shows we get about 2 times speed up.

For simple buffer object that doesn't need serialization, we still go with `DeclarativeAggregate`, to avoid performance regression.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16383 from cloud-fan/aggregator.
2016-12-26 22:10:20 +08:00
wangzhenhua 3cff816157 [SPARK-18911][SQL] Define CatalogStatistics to interact with metastore and convert it to Statistics in relations
## What changes were proposed in this pull request?

Statistics in LogicalPlan should use attributes to refer to columns rather than column names, because two columns from two relations can have the same column name. But CatalogTable doesn't have the concepts of attribute or broadcast hint in Statistics. Therefore, putting Statistics in CatalogTable is confusing.

We define a different statistic structure in CatalogTable, which is only responsible for interacting with metastore, and is converted to statistics in LogicalPlan when it is used.

## How was this patch tested?

add test cases

Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>

Closes #16323 from wzhfy/nameToAttr.
2016-12-24 15:34:44 +08:00
Reynold Xin 2615100055 [SPARK-18973][SQL] Remove SortPartitions and RedistributeData
## What changes were proposed in this pull request?
SortPartitions and RedistributeData logical operators are not actually used and can be removed. Note that we do have a Sort operator (with global flag false) that subsumed SortPartitions.

## How was this patch tested?
Also updated test cases to reflect the removal.

Author: Reynold Xin <rxin@databricks.com>

Closes #16381 from rxin/SPARK-18973.
2016-12-22 19:35:09 +01:00
Tathagata Das 83a6ace0d1 [SPARK-18234][SS] Made update mode public
## What changes were proposed in this pull request?

Made update mode public. As part of that here are the changes.
- Update DatastreamWriter to accept "update"
- Changed package of InternalOutputModes from o.a.s.sql to o.a.s.sql.catalyst
- Added update mode state removing with watermark to StateStoreSaveExec

## How was this patch tested?

Added new tests in changed modules

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #16360 from tdas/SPARK-18234.
2016-12-21 16:43:17 -08:00
Ryan Williams afd9bc1d8a [SPARK-17807][CORE] split test-tags into test-JAR
Remove spark-tag's compile-scope dependency (and, indirectly, spark-core's compile-scope transitive-dependency) on scalatest by splitting test-oriented tags into spark-tags' test JAR.

Alternative to #16303.

Author: Ryan Williams <ryan.blake.williams@gmail.com>

Closes #16311 from ryan-williams/tt.
2016-12-21 16:37:20 -08:00
Wenchen Fan f923c849e5 [SPARK-18899][SPARK-18912][SPARK-18913][SQL] refactor the error checking when append data to an existing table
## What changes were proposed in this pull request?

When we append data to an existing table with `DataFrameWriter.saveAsTable`, we will do various checks to make sure the appended data is consistent with the existing data.

However, we get the information of the existing table by matching the table relation, instead of looking at the table metadata. This is error-prone, e.g. we only check the number of columns for `HadoopFsRelation`, we forget to check bucketing, etc.

This PR refactors the error checking by looking at the metadata of the existing table, and fix several bugs:
* SPARK-18899: We forget to check if the specified bucketing matched the existing table, which may lead to a problematic table that has different bucketing in different data files.
* SPARK-18912: We forget to check the number of columns for non-file-based data source table
* SPARK-18913: We don't support append data to a table with special column names.

## How was this patch tested?
new regression test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16313 from cloud-fan/bug1.
2016-12-19 20:03:33 -08:00
jiangxingbo 70d495dcec [SPARK-18624][SQL] Implicit cast ArrayType(InternalType)
## What changes were proposed in this pull request?

Currently `ImplicitTypeCasts` doesn't handle casts between `ArrayType`s, this is not convenient, we should add a rule to enable casting from `ArrayType(InternalType)` to `ArrayType(newInternalType)`.

Goals:
1. Add a rule to `ImplicitTypeCasts` to enable casting between `ArrayType`s;
2. Simplify `Percentile` and `ApproximatePercentile`.

## How was this patch tested?

Updated test cases in `TypeCoercionSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #16057 from jiangxb1987/implicit-cast-complex-types.
2016-12-19 21:20:47 +01:00
Reynold Xin 172a52f5d3 [SPARK-18892][SQL] Alias percentile_approx approx_percentile
## What changes were proposed in this pull request?
percentile_approx is the name used in Hive, and approx_percentile is the name used in Presto. approx_percentile is actually more consistent with our approx_count_distinct. Given the cost to alias SQL functions is low (one-liner), it'd be better to just alias them so it is easier to use.

## How was this patch tested?
Technically I could add an end-to-end test to verify this one-line change, but it seemed too trivial to me.

Author: Reynold Xin <rxin@databricks.com>

Closes #16300 from rxin/SPARK-18892.
2016-12-15 21:58:27 -08:00
Tathagata Das 4f7292c875 [SPARK-18870] Disallowed Distinct Aggregations on Streaming Datasets
## What changes were proposed in this pull request?

Check whether Aggregation operators on a streaming subplan have aggregate expressions with isDistinct = true.

## How was this patch tested?

Added unit test

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #16289 from tdas/SPARK-18870.
2016-12-15 11:54:35 -08:00
jiangxingbo 01e14bf303 [SPARK-17910][SQL] Allow users to update the comment of a column
## What changes were proposed in this pull request?

Right now, once a user set the comment of a column with create table command, he/she cannot update the comment. It will be useful to provide a public interface (e.g. SQL) to do that.

This PR implements the following SQL statement:
```
ALTER TABLE table [PARTITION partition_spec]
CHANGE [COLUMN] column_old_name column_new_name column_dataType
[COMMENT column_comment]
[FIRST | AFTER column_name];
```

For further expansion, we could support alter `name`/`dataType`/`index` of a column too.

## How was this patch tested?

Add new test cases in `ExternalCatalogSuite` and `SessionCatalogSuite`.
Add sql file test for `ALTER TABLE CHANGE COLUMN` statement.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15717 from jiangxb1987/change-column.
2016-12-15 10:09:42 -08:00
Reynold Xin 5d510c693a [SPARK-18869][SQL] Add TreeNode.p that returns BaseType
## What changes were proposed in this pull request?
After the bug fix in SPARK-18854, TreeNode.apply now returns TreeNode[_] rather than a more specific type. It would be easier for interactive debugging to introduce a function that returns the BaseType.

## How was this patch tested?
N/A - this is a developer only feature used for interactive debugging. As long as it compiles, it should be good to go. I tested this in spark-shell.

Author: Reynold Xin <rxin@databricks.com>

Closes #16288 from rxin/SPARK-18869.
2016-12-14 21:08:45 -08:00
Reynold Xin ffdd1fcd1e [SPARK-18854][SQL] numberedTreeString and apply(i) inconsistent for subqueries
## What changes were proposed in this pull request?
This is a bug introduced by subquery handling. numberedTreeString (which uses generateTreeString under the hood) numbers trees including innerChildren (used to print subqueries), but apply (which uses getNodeNumbered) ignores innerChildren. As a result, apply(i) would return the wrong plan node if there are subqueries.

This patch fixes the bug.

## How was this patch tested?
Added a test case in SubquerySuite.scala to test both the depth-first traversal of numbering as well as making sure the two methods are consistent.

Author: Reynold Xin <rxin@databricks.com>

Closes #16277 from rxin/SPARK-18854.
2016-12-14 16:12:14 -08:00
Reynold Xin 5d79947369 [SPARK-18853][SQL] Project (UnaryNode) is way too aggressive in estimating statistics
## What changes were proposed in this pull request?
This patch reduces the default number element estimation for arrays and maps from 100 to 1. The issue with the 100 number is that when nested (e.g. an array of map), 100 * 100 would be used as the default size. This sounds like just an overestimation which doesn't seem that bad (since it is usually better to overestimate than underestimate). However, due to the way we assume the size output for Project (new estimated column size / old estimated column size), this overestimation can become underestimation. It is actually in general in this case safer to assume 1 default element.

## How was this patch tested?
This should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #16274 from rxin/SPARK-18853.
2016-12-14 21:22:49 +01:00
Nattavut Sutyanyong cccd64393e [SPARK-18814][SQL] CheckAnalysis rejects TPCDS query 32
## What changes were proposed in this pull request?
Move the checking of GROUP BY column in correlated scalar subquery from CheckAnalysis
to Analysis to fix a regression caused by SPARK-18504.

This problem can be reproduced with a simple script now.

Seq((1,1)).toDF("pk","pv").createOrReplaceTempView("p")
Seq((1,1)).toDF("ck","cv").createOrReplaceTempView("c")
sql("select * from p,c where p.pk=c.ck and c.cv = (select avg(c1.cv) from c c1 where c1.ck = p.pk)").show

The requirements are:
1. We need to reference the same table twice in both the parent and the subquery. Here is the table c.
2. We need to have a correlated predicate but to a different table. Here is from c (as c1) in the subquery to p in the parent.
3. We will then "deduplicate" c1.ck in the subquery to `ck#<n1>#<n2>` at `Project` above `Aggregate` of `avg`. Then when we compare `ck#<n1>#<n2>` and the original group by column `ck#<n1>` by their canonicalized form, which is #<n2> != #<n1>. That's how we trigger the exception added in SPARK-18504.

## How was this patch tested?

SubquerySuite and a simplified version of TPCDS-Q32

Author: Nattavut Sutyanyong <nsy.can@gmail.com>

Closes #16246 from nsyca/18814.
2016-12-14 11:09:31 +01:00
Wenchen Fan 3e307b4959 [SPARK-18566][SQL] remove OverwriteOptions
## What changes were proposed in this pull request?

`OverwriteOptions` was introduced in https://github.com/apache/spark/pull/15705, to carry the information of static partitions. However, after further refactor, this information becomes duplicated and we can remove `OverwriteOptions`.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15995 from cloud-fan/overwrite.
2016-12-14 11:30:34 +08:00
Marcelo Vanzin 3ae63b808a [SPARK-18752][SQL] Follow-up: add scaladoc explaining isSrcLocal arg.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #16257 from vanzin/SPARK-18752.2.
2016-12-13 17:55:38 -08:00
jiangxingbo 5572ccf86b [SPARK-17932][SQL][FOLLOWUP] Change statement SHOW TABLES EXTENDED to SHOW TABLE EXTENDED
## What changes were proposed in this pull request?

Change the statement `SHOW TABLES [EXTENDED] [(IN|FROM) database_name] [[LIKE] 'identifier_with_wildcards'] [PARTITION(partition_spec)]` to the following statements:

- SHOW TABLES [(IN|FROM) database_name] [[LIKE] 'identifier_with_wildcards']
- SHOW TABLE EXTENDED [(IN|FROM) database_name] LIKE 'identifier_with_wildcards' [PARTITION(partition_spec)]

After this change, the statements `SHOW TABLE/SHOW TABLES` have the same syntax with that HIVE has.

## How was this patch tested?
Modified the test sql file `show-tables.sql`;
Modified the test suite `DDLSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #16262 from jiangxb1987/show-table-extended.
2016-12-13 19:04:34 +01:00
Marcelo Vanzin f280ccf449 [SPARK-18835][SQL] Don't expose Guava types in the JavaTypeInference API.
This avoids issues during maven tests because of shading.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #16260 from vanzin/SPARK-18835.
2016-12-13 10:02:19 -08:00
Andrew Ray 46d30ac484 [SPARK-18717][SQL] Make code generation for Scala Map work with immutable.Map also
## What changes were proposed in this pull request?

Fixes compile errors in generated code when user has case class with a `scala.collections.immutable.Map` instead of a `scala.collections.Map`. Since ArrayBasedMapData.toScalaMap returns the immutable version we can make it work with both.

## How was this patch tested?

Additional unit tests.

Author: Andrew Ray <ray.andrew@gmail.com>

Closes #16161 from aray/fix-map-codegen.
2016-12-13 15:49:22 +08:00
Marcelo Vanzin 476b34c23a [SPARK-18752][HIVE] isSrcLocal" value should be set from user query.
The value of the "isSrcLocal" parameter passed to Hive's loadTable and
loadPartition methods needs to be set according to the user query (e.g.
"LOAD DATA LOCAL"), and not the current code that tries to guess what
it should be.

For existing versions of Hive the current behavior is probably ok, but
some recent changes in the Hive code changed the semantics slightly,
making code that sets "isSrcLocal" to "true" incorrectly to do the
wrong thing. It would end up moving the parent directory of the files
into the final location, instead of the file themselves, resulting
in a table that cannot be read.

I modified HiveCommandSuite so that existing "LOAD DATA" tests are run
both in local and non-local mode, since the semantics are slightly different.
The tests include a few new checks to make sure the semantics follow
what Hive describes in its documentation.

Tested with existing unit tests and also ran some Hive integration tests
with a version of Hive containing the changes that surfaced the problem.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #16179 from vanzin/SPARK-18752.
2016-12-12 14:19:42 -08:00
Wenchen Fan 9abd05b6b9
[SQL][MINOR] simplify a test to fix the maven tests
## What changes were proposed in this pull request?

After https://github.com/apache/spark/pull/15620 , all of the Maven-based 2.0 Jenkins jobs time out consistently. As I pointed out in https://github.com/apache/spark/pull/15620#discussion_r91829129 , it seems that the regression test is an overkill and may hit constants pool size limitation, which is a known issue and hasn't been fixed yet.

Since #15620 only fix the code size limitation problem, we can simplify the test to avoid hitting constants pool size limitation.

## How was this patch tested?

test only change

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16244 from cloud-fan/minor.
2016-12-11 09:12:46 +00:00
wangzhenhua a29ee55aaa [SPARK-18815][SQL] Fix NPE when collecting column stats for string/binary column having only null values
## What changes were proposed in this pull request?

During column stats collection, average and max length will be null if a column of string/binary type has only null values. To fix this, I use default size when avg/max length is null.

## How was this patch tested?

Add a test for handling null columns

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #16243 from wzhfy/nullStats.
2016-12-10 21:25:29 -08:00
Huaxin Gao c5172568b5 [SPARK-17460][SQL] Make sure sizeInBytes in Statistics will not overflow
## What changes were proposed in this pull request?

1. In SparkStrategies.canBroadcast, I will add the check   plan.statistics.sizeInBytes >= 0
2. In LocalRelations.statistics, when calculate the statistics, I will change the size to BigInt so it won't overflow.

## How was this patch tested?

I will add a test case to make sure the statistics.sizeInBytes won't overflow.

Author: Huaxin Gao <huaxing@us.ibm.com>

Closes #16175 from huaxingao/spark-17460.
2016-12-10 22:41:40 +08:00
Jacek Laskowski b162cc0c28
[MINOR][CORE][SQL][DOCS] Typo fixes
## What changes were proposed in this pull request?

Typo fixes

## How was this patch tested?

Local build. Awaiting the official build.

Author: Jacek Laskowski <jacek@japila.pl>

Closes #16144 from jaceklaskowski/typo-fixes.
2016-12-09 18:45:57 +08:00
Nathan Howell bec0a9217b [SPARK-18654][SQL] Remove unreachable patterns in makeRootConverter
## What changes were proposed in this pull request?

`makeRootConverter` is only called with a `StructType` value. By making this method less general we can remove pattern matches, which are never actually hit outside of the test suite.

## How was this patch tested?

The existing tests.

Author: Nathan Howell <nhowell@godaddy.com>

Closes #16084 from NathanHowell/SPARK-18654.
2016-12-07 16:52:05 -08:00
Andrew Ray f1fca81b16 [SPARK-17760][SQL] AnalysisException with dataframe pivot when groupBy column is not attribute
## What changes were proposed in this pull request?

Fixes AnalysisException for pivot queries that have group by columns that are expressions and not attributes by substituting the expressions output attribute in the second aggregation and final projection.

## How was this patch tested?

existing and additional unit tests

Author: Andrew Ray <ray.andrew@gmail.com>

Closes #16177 from aray/SPARK-17760.
2016-12-07 04:44:14 -08:00
Herman van Hovell 381ef4ea76 [SPARK-18634][SQL][TRIVIAL] Touch-up Generate
## What changes were proposed in this pull request?
I jumped the gun on merging https://github.com/apache/spark/pull/16120, and missed a tiny potential problem. This PR fixes that by changing a val into a def; this should prevent potential serialization/initialization weirdness from happening.

## How was this patch tested?
Existing tests.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #16170 from hvanhovell/SPARK-18634.
2016-12-06 05:51:39 -08:00
Michael Allman 772ddbeaa6 [SPARK-18572][SQL] Add a method listPartitionNames to ExternalCatalog
(Link to Jira issue: https://issues.apache.org/jira/browse/SPARK-18572)

## What changes were proposed in this pull request?

Currently Spark answers the `SHOW PARTITIONS` command by fetching all of the table's partition metadata from the external catalog and constructing partition names therefrom. The Hive client has a `getPartitionNames` method which is many times faster for this purpose, with the performance improvement scaling with the number of partitions in a table.

To test the performance impact of this PR, I ran the `SHOW PARTITIONS` command on two Hive tables with large numbers of partitions. One table has ~17,800 partitions, and the other has ~95,000 partitions. For the purposes of this PR, I'll call the former table `table1` and the latter table `table2`. I ran 5 trials for each table with before-and-after versions of this PR. The results are as follows:

Spark at bdc8153, `SHOW PARTITIONS table1`, times in seconds:
7.901
3.983
4.018
4.331
4.261

Spark at bdc8153, `SHOW PARTITIONS table2`
(Timed out after 10 minutes with a `SocketTimeoutException`.)

Spark at this PR, `SHOW PARTITIONS table1`, times in seconds:
3.801
0.449
0.395
0.348
0.336

Spark at this PR, `SHOW PARTITIONS table2`, times in seconds:
5.184
1.63
1.474
1.519
1.41

Taking the best times from each trial, we get a 12x performance improvement for a table with ~17,800 partitions and at least a 426x improvement for a table with ~95,000 partitions. More significantly, the latter command doesn't even complete with the current code in master.

This is actually a patch we've been using in-house at VideoAmp since Spark 1.1. It's made all the difference in the practical usability of our largest tables. Even with tables with about 1,000 partitions there's a performance improvement of about 2-3x.

## How was this patch tested?

I added a unit test to `VersionsSuite` which tests that the Hive client's `getPartitionNames` method returns the correct number of partitions.

Author: Michael Allman <michael@videoamp.com>

Closes #15998 from mallman/spark-18572-list_partition_names.
2016-12-06 11:33:35 +08:00
Liang-Chi Hsieh 3ba69b6485 [SPARK-18634][PYSPARK][SQL] Corruption and Correctness issues with exploding Python UDFs
## What changes were proposed in this pull request?

As reported in the Jira, there are some weird issues with exploding Python UDFs in SparkSQL.

The following test code can reproduce it. Notice: the following test code is reported to return wrong results in the Jira. However, as I tested on master branch, it causes exception and so can't return any result.

    >>> from pyspark.sql.functions import *
    >>> from pyspark.sql.types import *
    >>>
    >>> df = spark.range(10)
    >>>
    >>> def return_range(value):
    ...   return [(i, str(i)) for i in range(value - 1, value + 1)]
    ...
    >>> range_udf = udf(return_range, ArrayType(StructType([StructField("integer_val", IntegerType()),
    ...                                                     StructField("string_val", StringType())])))
    >>>
    >>> df.select("id", explode(range_udf(df.id))).show()
    Traceback (most recent call last):
      File "<stdin>", line 1, in <module>
      File "/spark/python/pyspark/sql/dataframe.py", line 318, in show
        print(self._jdf.showString(n, 20))
      File "/spark/python/lib/py4j-0.10.4-src.zip/py4j/java_gateway.py", line 1133, in __call__
      File "/spark/python/pyspark/sql/utils.py", line 63, in deco
        return f(*a, **kw)
      File "/spark/python/lib/py4j-0.10.4-src.zip/py4j/protocol.py", line 319, in get_return_value py4j.protocol.Py4JJavaError: An error occurred while calling o126.showString.: java.lang.AssertionError: assertion failed
        at scala.Predef$.assert(Predef.scala:156)
        at org.apache.spark.sql.execution.CodegenSupport$class.consume(WholeStageCodegenExec.scala:120)
        at org.apache.spark.sql.execution.GenerateExec.consume(GenerateExec.scala:57)

The cause of this issue is, in `ExtractPythonUDFs` we insert `BatchEvalPythonExec` to run PythonUDFs in batch. `BatchEvalPythonExec` will add extra outputs (e.g., `pythonUDF0`) to original plan. In above case, the original `Range` only has one output `id`. After `ExtractPythonUDFs`, the added `BatchEvalPythonExec` has two outputs `id` and `pythonUDF0`.

Because the output of `GenerateExec` is given after analysis phase, in above case, it is the combination of `id`, i.e., the output of `Range`, and `col`. But in planning phase, we change `GenerateExec`'s child plan to `BatchEvalPythonExec` with additional output attributes.

It will cause no problem in non wholestage codegen. Because when evaluating the additional attributes are projected out the final output of `GenerateExec`.

However, as `GenerateExec` now supports wholestage codegen, the framework will input all the outputs of the child plan to `GenerateExec`. Then when consuming `GenerateExec`'s output data (i.e., calling `consume`), the number of output attributes is different to the output variables in wholestage codegen.

To solve this issue, this patch only gives the generator's output to `GenerateExec` after analysis phase. `GenerateExec`'s output is the combination of its child plan's output and the generator's output. So when we change `GenerateExec`'s child, its output is still correct.

## How was this patch tested?

Added test cases to PySpark.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #16120 from viirya/fix-py-udf-with-generator.
2016-12-05 17:50:43 -08:00
Wenchen Fan 01a7d33d08 [SPARK-18711][SQL] should disable subexpression elimination for LambdaVariable
## What changes were proposed in this pull request?

This is kind of a long-standing bug, it's hidden until https://github.com/apache/spark/pull/15780 , which may add `AssertNotNull` on top of `LambdaVariable` and thus enables subexpression elimination.

However, subexpression elimination will evaluate the common expressions at the beginning, which is invalid for `LambdaVariable`. `LambdaVariable` usually represents loop variable, which can't be evaluated ahead of the loop.

This PR skips expressions containing `LambdaVariable` when doing subexpression elimination.

## How was this patch tested?

updated test in `DatasetAggregatorSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16143 from cloud-fan/aggregator.
2016-12-05 11:37:13 -08:00
Reynold Xin e9730b707d [SPARK-18702][SQL] input_file_block_start and input_file_block_length
## What changes were proposed in this pull request?
We currently have function input_file_name to get the path of the input file, but don't have functions to get the block start offset and length. This patch introduces two functions:

1. input_file_block_start: returns the file block start offset, or -1 if not available.

2. input_file_block_length: returns the file block length, or -1 if not available.

## How was this patch tested?
Updated existing test cases in ColumnExpressionSuite that covered input_file_name to also cover the two new functions.

Author: Reynold Xin <rxin@databricks.com>

Closes #16133 from rxin/SPARK-18702.
2016-12-04 21:51:10 -08:00
Kapil Singh e463678b19 [SPARK-18091][SQL] Deep if expressions cause Generated SpecificUnsafeProjection code to exceed JVM code size limit
## What changes were proposed in this pull request?

Fix for SPARK-18091 which is a bug related to large if expressions causing generated SpecificUnsafeProjection code to exceed JVM code size limit.

This PR changes if expression's code generation to place its predicate, true value and false value expressions' generated code in separate methods in context so as to never generate too long combined code.
## How was this patch tested?

Added a unit test and also tested manually with the application (having transformations similar to the unit test) which caused the issue to be identified in the first place.

Author: Kapil Singh <kapsingh@adobe.com>

Closes #15620 from kapilsingh5050/SPARK-18091-IfCodegenFix.
2016-12-04 17:16:40 +08:00
Nattavut Sutyanyong 4a3c09601b [SPARK-18582][SQL] Whitelist LogicalPlan operators allowed in correlated subqueries
## What changes were proposed in this pull request?

This fix puts an explicit list of operators that Spark supports for correlated subqueries.

## How was this patch tested?

Run sql/test, catalyst/test and add a new test case on Generate.

Author: Nattavut Sutyanyong <nsy.can@gmail.com>

Closes #16046 from nsyca/spark18455.0.
2016-12-03 11:36:26 -08:00
Reynold Xin c7c7265950 [SPARK-18695] Bump master branch version to 2.2.0-SNAPSHOT
## What changes were proposed in this pull request?
This patch bumps master branch version to 2.2.0-SNAPSHOT.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #16126 from rxin/SPARK-18695.
2016-12-02 21:09:37 -08:00
Ryan Blue 48778976e0 [SPARK-18677] Fix parsing ['key'] in JSON path expressions.
## What changes were proposed in this pull request?

This fixes the parser rule to match named expressions, which doesn't work for two reasons:
1. The name match is not coerced to a regular expression (missing .r)
2. The surrounding literals are incorrect and attempt to escape a single quote, which is unnecessary

## How was this patch tested?

This adds test cases for named expressions using the bracket syntax, including one with quoted spaces.

Author: Ryan Blue <blue@apache.org>

Closes #16107 from rdblue/SPARK-18677-fix-json-path.
2016-12-02 08:41:40 -08:00
gatorsmile 2f8776ccad [SPARK-18674][SQL][FOLLOW-UP] improve the error message of using join
### What changes were proposed in this pull request?
Added a test case for using joins with nested fields.

### How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16110 from gatorsmile/followup-18674.
2016-12-02 22:12:19 +08:00
Eric Liang 7935c8470c [SPARK-18659][SQL] Incorrect behaviors in overwrite table for datasource tables
## What changes were proposed in this pull request?

Two bugs are addressed here
1. INSERT OVERWRITE TABLE sometime crashed when catalog partition management was enabled. This was because when dropping partitions after an overwrite operation, the Hive client will attempt to delete the partition files. If the entire partition directory was dropped, this would fail. The PR fixes this by adding a flag to control whether the Hive client should attempt to delete files.
2. The static partition spec for OVERWRITE TABLE was not correctly resolved to the case-sensitive original partition names. This resulted in the entire table being overwritten if you did not correctly capitalize your partition names.

cc yhuai cloud-fan

## How was this patch tested?

Unit tests. Surprisingly, the existing overwrite table tests did not catch these edge cases.

Author: Eric Liang <ekl@databricks.com>

Closes #16088 from ericl/spark-18659.
2016-12-02 21:59:02 +08:00
Nathan Howell c82f16c15e [SPARK-18658][SQL] Write text records directly to a FileOutputStream
## What changes were proposed in this pull request?

This replaces uses of `TextOutputFormat` with an `OutputStream`, which will either write directly to the filesystem or indirectly via a compressor (if so configured). This avoids intermediate buffering.

The inverse of this (reading directly from a stream) is necessary for streaming large JSON records (when `wholeFile` is enabled) so I wanted to keep the read and write paths symmetric.

## How was this patch tested?

Existing unit tests.

Author: Nathan Howell <nhowell@godaddy.com>

Closes #16089 from NathanHowell/SPARK-18658.
2016-12-01 21:40:49 -08:00
Reynold Xin d3c90b74ed [SPARK-18663][SQL] Simplify CountMinSketch aggregate implementation
## What changes were proposed in this pull request?
SPARK-18429 introduced count-min sketch aggregate function for SQL, but the implementation and testing is more complicated than needed. This simplifies the test cases and removes support for data types that don't have clear equality semantics:

1. Removed support for floating point and decimal types.

2. Removed the heavy randomized tests. The underlying CountMinSketch implementation already had pretty good test coverage through randomized tests, and the SPARK-18429 implementation is just to add an aggregate function wrapper around CountMinSketch. There is no need for randomized tests at three different levels of the implementations.

## How was this patch tested?
A lot of the change is to simplify test cases.

Author: Reynold Xin <rxin@databricks.com>

Closes #16093 from rxin/SPARK-18663.
2016-12-01 21:38:52 -08:00
Kazuaki Ishizaki 38b9e69623 [SPARK-18284][SQL] Make ExpressionEncoder.serializer.nullable precise
## What changes were proposed in this pull request?

This PR makes `ExpressionEncoder.serializer.nullable` for flat encoder for a primitive type `false`. Since it is `true` for now, it is too conservative.
While `ExpressionEncoder.schema` has correct information (e.g. `<IntegerType, false>`), `serializer.head.nullable` of `ExpressionEncoder`, which got from `encoderFor[T]`, is always false. It is too conservative.

This is accomplished by checking whether a type is one of primitive types. If it is `true`, `nullable` should be `false`.

## How was this patch tested?

Added new tests for encoder and dataframe

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #15780 from kiszk/SPARK-18284.
2016-12-02 12:30:13 +08:00
Wenchen Fan e653484710 [SPARK-18674][SQL] improve the error message of using join
## What changes were proposed in this pull request?

The current error message of USING join is quite confusing, for example:
```
scala> val df1 = List(1,2,3).toDS.withColumnRenamed("value", "c1")
df1: org.apache.spark.sql.DataFrame = [c1: int]

scala> val df2 = List(1,2,3).toDS.withColumnRenamed("value", "c2")
df2: org.apache.spark.sql.DataFrame = [c2: int]

scala> df1.join(df2, usingColumn = "c1")
org.apache.spark.sql.AnalysisException: using columns ['c1] can not be resolved given input columns: [c1, c2] ;;
'Join UsingJoin(Inner,List('c1))
:- Project [value#1 AS c1#3]
:  +- LocalRelation [value#1]
+- Project [value#7 AS c2#9]
   +- LocalRelation [value#7]
```

after this PR, it becomes:
```
scala> val df1 = List(1,2,3).toDS.withColumnRenamed("value", "c1")
df1: org.apache.spark.sql.DataFrame = [c1: int]

scala> val df2 = List(1,2,3).toDS.withColumnRenamed("value", "c2")
df2: org.apache.spark.sql.DataFrame = [c2: int]

scala> df1.join(df2, usingColumn = "c1")
org.apache.spark.sql.AnalysisException: USING column `c1` can not be resolved with the right join side, the right output is: [c2];
```

## How was this patch tested?

updated tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16100 from cloud-fan/natural.
2016-12-01 11:53:12 -08:00
Eric Liang 88f559f20a [SPARK-18635][SQL] Partition name/values not escaped correctly in some cases
## What changes were proposed in this pull request?

Due to confusion between URI vs paths, in certain cases we escape partition values too many times, which causes some Hive client operations to fail or write data to the wrong location. This PR fixes at least some of these cases.

To my understanding this is how values, filesystem paths, and URIs interact.
- Hive stores raw (unescaped) partition values that are returned to you directly when you call listPartitions.
- Internally, we convert these raw values to filesystem paths via `ExternalCatalogUtils.[un]escapePathName`.
- In some circumstances we store URIs instead of filesystem paths. When a path is converted to a URI via `path.toURI`, the escaped partition values are further URI-encoded. This means that to get a path back from a URI, you must call `new Path(new URI(uriTxt))` in order to decode the URI-encoded string.
- In `CatalogStorageFormat` we store URIs as strings. This makes it easy to forget to URI-decode the value before converting it into a path.
- Finally, the Hive client itself uses mostly Paths for representing locations, and only URIs occasionally.

In the future we should probably clean this up, perhaps by dropping use of URIs when unnecessary. We should also try fixing escaping for partition names as well as values, though names are unlikely to contain special characters.

cc mallman cloud-fan yhuai

## How was this patch tested?

Unit tests.

Author: Eric Liang <ekl@databricks.com>

Closes #16071 from ericl/spark-18635.
2016-12-01 16:48:10 +08:00
Wenchen Fan f135b70fd5 [SPARK-18251][SQL] the type of Dataset can't be Option of non-flat type
## What changes were proposed in this pull request?

For input object of non-flat type, we can't encode it to row if it's null, as Spark SQL doesn't allow the entire row to be null, only its columns can be null. That's the reason we forbid users to use top level null objects in https://github.com/apache/spark/pull/13469

However, if users wrap non-flat type with `Option`, then we may still encoder top level null object to row, which is not allowed.

This PR fixes this case, and suggests users to wrap their type with `Tuple1` if they do wanna top level null objects.

## How was this patch tested?

new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15979 from cloud-fan/option.
2016-11-30 13:36:17 -08:00
jiangxingbo c24076dcf8 [SPARK-17932][SQL] Support SHOW TABLES EXTENDED LIKE 'identifier_with_wildcards' statement
## What changes were proposed in this pull request?

Currently we haven't implemented `SHOW TABLE EXTENDED` in Spark 2.0. This PR is to implement the statement.
Goals:
1. Support `SHOW TABLES EXTENDED LIKE 'identifier_with_wildcards'`;
2. Explicitly output an unsupported error message for `SHOW TABLES [EXTENDED] ... PARTITION` statement;
3. Improve test cases for `SHOW TABLES` statement.

## How was this patch tested?
1. Add new test cases in file `show-tables.sql`.
2. Modify tests for `SHOW TABLES` in `DDLSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15958 from jiangxb1987/show-table-extended.
2016-11-30 03:59:25 -08:00
gatorsmile 2eb093decb [SPARK-17897][SQL] Fixed IsNotNull Constraint Inference Rule
### What changes were proposed in this pull request?
The `constraints` of an operator is the expressions that evaluate to `true` for all the rows produced. That means, the expression result should be neither `false` nor `unknown` (NULL). Thus, we can conclude that `IsNotNull` on all the constraints, which are generated by its own predicates or propagated from the children. The constraint can be a complex expression. For better usage of these constraints, we try to push down `IsNotNull` to the lowest-level expressions (i.e., `Attribute`). `IsNotNull` can be pushed through an expression when it is null intolerant. (When the input is NULL, the null-intolerant expression always evaluates to NULL.)

Below is the existing code we have for `IsNotNull` pushdown.
```Scala
  private def scanNullIntolerantExpr(expr: Expression): Seq[Attribute] = expr match {
    case a: Attribute => Seq(a)
    case _: NullIntolerant | IsNotNull(_: NullIntolerant) =>
      expr.children.flatMap(scanNullIntolerantExpr)
    case _ => Seq.empty[Attribute]
  }
```

**`IsNotNull` itself is not null-intolerant.** It converts `null` to `false`. If the expression does not include any `Not`-like expression, it works; otherwise, it could generate a wrong result. This PR is to fix the above function by removing the `IsNotNull` from the inference. After the fix, when a constraint has a `IsNotNull` expression, we infer new attribute-specific `IsNotNull` constraints if and only if `IsNotNull` appears in the root.

Without the fix, the following test case will return empty.
```Scala
val data = Seq[java.lang.Integer](1, null).toDF("key")
data.filter("not key is not null").show()
```
Before the fix, the optimized plan is like
```
== Optimized Logical Plan ==
Project [value#1 AS key#3]
+- Filter (isnotnull(value#1) && NOT isnotnull(value#1))
   +- LocalRelation [value#1]
```

After the fix, the optimized plan is like
```
== Optimized Logical Plan ==
Project [value#1 AS key#3]
+- Filter NOT isnotnull(value#1)
   +- LocalRelation [value#1]
```

### How was this patch tested?
Added a test

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16067 from gatorsmile/isNotNull2.
2016-11-30 19:40:58 +08:00
Herman van Hovell 879ba71110 [SPARK-18622][SQL] Fix the datatype of the Sum aggregate function
## What changes were proposed in this pull request?
The result of a `sum` aggregate function is typically a Decimal, Double or a Long. Currently the output dataType is based on input's dataType.

The `FunctionArgumentConversion` rule will make sure that the input is promoted to the largest type, and that also ensures that the output uses a (hopefully) sufficiently large output dataType. The issue is that sum is in a resolved state when we cast the input type, this means that rules assuming that the dataType of the expression does not change anymore could have been applied in the mean time. This is what happens if we apply `WidenSetOperationTypes` before applying the casts, and this breaks analysis.

The most straight forward and future proof solution is to make `sum` always output the widest dataType in its class (Long for IntegralTypes, Decimal for DecimalTypes & Double for FloatType and DoubleType). This PR implements that solution.

We should move expression specific type casting rules into the given Expression at some point.

## How was this patch tested?
Added (regression) tests to SQLQueryTestSuite's `union.sql`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #16063 from hvanhovell/SPARK-18622.
2016-11-30 15:25:33 +08:00
Herman van Hovell af9789a4f5 [SPARK-18632][SQL] AggregateFunction should not implement ImplicitCastInputTypes
## What changes were proposed in this pull request?
`AggregateFunction` currently implements `ImplicitCastInputTypes` (which enables implicit input type casting). There are actually quite a few situations in which we don't need this, or require more control over our input. A recent example is the aggregate for `CountMinSketch` which should only take string, binary or integral types inputs.

This PR removes `ImplicitCastInputTypes` from the `AggregateFunction` and makes a case-by-case decision on what kind of input validation we should use.

## How was this patch tested?
Refactoring only. Existing tests.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #16066 from hvanhovell/SPARK-18632.
2016-11-29 20:05:15 -08:00
Nattavut Sutyanyong 3600635215 [SPARK-18614][SQL] Incorrect predicate pushdown from ExistenceJoin
## What changes were proposed in this pull request?

ExistenceJoin should be treated the same as LeftOuter and LeftAnti, not InnerLike and LeftSemi. This is not currently exposed because the rewrite of [NOT] EXISTS OR ... to ExistenceJoin happens in rule RewritePredicateSubquery, which is in a separate rule set and placed after the rule PushPredicateThroughJoin. During the transformation in the rule PushPredicateThroughJoin, an ExistenceJoin never exists.

The semantics of ExistenceJoin says we need to preserve all the rows from the left table through the join operation as if it is a regular LeftOuter join. The ExistenceJoin augments the LeftOuter operation with a new column called exists, set to true when the join condition in the ON clause is true and false otherwise. The filter of any rows will happen in the Filter operation above the ExistenceJoin.

Example:

A(c1, c2): { (1, 1), (1, 2) }
// B can be any value as it is irrelevant in this example
B(c1): { (NULL) }

select A.*
from   A
where  exists (select 1 from B where A.c1 = A.c2)
       or A.c2=2

In this example, the correct result is all the rows from A. If the pattern ExistenceJoin around line 935 in Optimizer.scala is indeed active, the code will push down the predicate A.c1 = A.c2 to be a Filter on relation A, which will incorrectly filter the row (1,2) from A.

## How was this patch tested?

Since this is not an exposed case, no new test cases is added. The scenario is discovered via a code review of another PR and confirmed to be valid with peer.

Author: Nattavut Sutyanyong <nsy.can@gmail.com>

Closes #16044 from nsyca/spark-18614.
2016-11-29 15:27:43 -08:00
wangzhenhua d57a594b8b [SPARK-18429][SQL] implement a new Aggregate for CountMinSketch
## What changes were proposed in this pull request?

This PR implements a new Aggregate to generate count min sketch, which is a wrapper of CountMinSketch.

## How was this patch tested?

add test cases

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #15877 from wzhfy/cms.
2016-11-29 13:16:46 -08:00
hyukjinkwon 1a870090e4
[SPARK-18615][DOCS] Switch to multi-line doc to avoid a genjavadoc bug for backticks
## What changes were proposed in this pull request?

Currently, single line comment does not mark down backticks to `<code>..</code>` but prints as they are (`` `..` ``). For example, the line below:

```scala
/** Return an RDD with the pairs from `this` whose keys are not in `other`. */
```

So, we could work around this as below:

```scala
/**
 * Return an RDD with the pairs from `this` whose keys are not in `other`.
 */
```

- javadoc

  - **Before**
    ![2016-11-29 10 39 14](https://cloud.githubusercontent.com/assets/6477701/20693606/e64c8f90-b622-11e6-8dfc-4a029216e23d.png)

  - **After**
    ![2016-11-29 10 39 08](https://cloud.githubusercontent.com/assets/6477701/20693607/e7280d36-b622-11e6-8502-d2e21cd5556b.png)

- scaladoc (this one looks fine either way)

  - **Before**
    ![2016-11-29 10 38 22](https://cloud.githubusercontent.com/assets/6477701/20693640/12c18aa8-b623-11e6-901a-693e2f6f8066.png)

  - **After**
    ![2016-11-29 10 40 05](https://cloud.githubusercontent.com/assets/6477701/20693642/14eb043a-b623-11e6-82ac-7cd0000106d1.png)

I suspect this is related with SPARK-16153 and genjavadoc issue in ` typesafehub/genjavadoc#85`.

## How was this patch tested?

I found them via

```
grep -r "\/\*\*.*\`" . | grep .scala
````

and then checked if each is in the public API documentation with manually built docs (`jekyll build`) with Java 7.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16050 from HyukjinKwon/javadoc-markdown.
2016-11-29 13:50:24 +00:00
hyukjinkwon f830bb9170
[SPARK-3359][DOCS] Make javadoc8 working for unidoc/genjavadoc compatibility in Java API documentation
## What changes were proposed in this pull request?

This PR make `sbt unidoc` complete with Java 8.

This PR roughly includes several fixes as below:

- Fix unrecognisable class and method links in javadoc by changing it from `[[..]]` to `` `...` ``

  ```diff
  - * A column that will be computed based on the data in a [[DataFrame]].
  + * A column that will be computed based on the data in a `DataFrame`.
  ```

- Fix throws annotations so that they are recognisable in javadoc

- Fix URL links to `<a href="http..."></a>`.

  ```diff
  - * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] model for regression.
  + * <a href="http://en.wikipedia.org/wiki/Decision_tree_learning">
  + * Decision tree (Wikipedia)</a> model for regression.
  ```

  ```diff
  -   * see http://en.wikipedia.org/wiki/Receiver_operating_characteristic
  +   * see <a href="http://en.wikipedia.org/wiki/Receiver_operating_characteristic">
  +   * Receiver operating characteristic (Wikipedia)</a>
  ```

- Fix < to > to

  - `greater than`/`greater than or equal to` or `less than`/`less than or equal to` where applicable.

  - Wrap it with `{{{...}}}` to print them in javadoc or use `{code ...}` or `{literal ..}`. Please refer https://github.com/apache/spark/pull/16013#discussion_r89665558

- Fix `</p>` complaint

## How was this patch tested?

Manually tested by `jekyll build` with Java 7 and 8

```
java version "1.7.0_80"
Java(TM) SE Runtime Environment (build 1.7.0_80-b15)
Java HotSpot(TM) 64-Bit Server VM (build 24.80-b11, mixed mode)
```

```
java version "1.8.0_45"
Java(TM) SE Runtime Environment (build 1.8.0_45-b14)
Java HotSpot(TM) 64-Bit Server VM (build 25.45-b02, mixed mode)
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16013 from HyukjinKwon/SPARK-3359-errors-more.
2016-11-29 09:41:32 +00:00
Tyson Condie 3c0beea475 [SPARK-18339][SPARK-18513][SQL] Don't push down current_timestamp for filters in StructuredStreaming and persist batch and watermark timestamps to offset log.
## What changes were proposed in this pull request?

For the following workflow:
1. I have a column called time which is at minute level precision in a Streaming DataFrame
2. I want to perform groupBy time, count
3. Then I want my MemorySink to only have the last 30 minutes of counts and I perform this by
.where('time >= current_timestamp().cast("long") - 30 * 60)
what happens is that the `filter` gets pushed down before the aggregation, and the filter happens on the source data for the aggregation instead of the result of the aggregation (where I actually want to filter).
I guess the main issue here is that `current_timestamp` is non-deterministic in the streaming context and shouldn't be pushed down the filter.
Does this require us to store the `current_timestamp` for each trigger of the streaming job, that is something to discuss.

Furthermore, we want to persist current batch timestamp and watermark timestamp to the offset log so that these values are consistent across multiple executions of the same batch.

brkyvz zsxwing tdas

## How was this patch tested?

A test was added to StreamingAggregationSuite ensuring the above use case is handled. The test injects a stream of time values (in seconds) to a query that runs in complete mode and only outputs the (count) aggregation results for the past 10 seconds.

Author: Tyson Condie <tcondie@gmail.com>

Closes #15949 from tcondie/SPARK-18339.
2016-11-28 23:07:17 -08:00
Herman van Hovell d449988b88 [SPARK-18058][SQL][TRIVIAL] Use dataType.sameResult(...) instead equality on asNullable datatypes
## What changes were proposed in this pull request?
This is absolutely minor. PR https://github.com/apache/spark/pull/15595 uses `dt1.asNullable == dt2.asNullable` expressions in a few places. It is however more efficient to call `dt1.sameType(dt2)`. I have replaced every instance of the first pattern with the second pattern (3/5 were introduced by #15595).

## How was this patch tested?
Existing tests.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #16041 from hvanhovell/SPARK-18058.
2016-11-28 21:43:33 -08:00
Shuai Lin e64a2047ea [SPARK-16282][SQL] Follow-up: remove "percentile" from temp function detection after implementing it natively
## What changes were proposed in this pull request?

In #15764 we added a mechanism to detect if a function is temporary or not. Hive functions are treated as non-temporary. Of the three hive functions, now "percentile" has been implemented natively, and "hash" has been removed. So we should update the list.

## How was this patch tested?

Unit tests.

Author: Shuai Lin <linshuai2012@gmail.com>

Closes #16049 from lins05/update-temp-function-detect-hive-list.
2016-11-28 20:23:48 -08:00
jiangxingbo 0f5f52a3d1 [SPARK-16282][SQL] Implement percentile SQL function.
## What changes were proposed in this pull request?

Implement percentile SQL function. It computes the exact percentile(s) of expr at pc with range in [0, 1].

## How was this patch tested?

Add a new testsuite `PercentileSuite` to test percentile directly.
Updated related testcases in `ExpressionToSQLSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>
Author: 蒋星博 <jiangxingbo@meituan.com>
Author: jiangxingbo <jiangxingbo@meituan.com>

Closes #14136 from jiangxb1987/percentile.
2016-11-28 11:05:58 -08:00
Yin Huai eba727757e [SPARK-18602] Set the version of org.codehaus.janino:commons-compiler to 3.0.0 to match the version of org.codehaus.janino:janino
## What changes were proposed in this pull request?
org.codehaus.janino:janino depends on org.codehaus.janino:commons-compiler and we have been upgraded to org.codehaus.janino:janino 3.0.0.

However, seems we are still pulling in org.codehaus.janino:commons-compiler 2.7.6 because of calcite. It looks like an accident because we exclude janino from calcite (see here https://github.com/apache/spark/blob/branch-2.1/pom.xml#L1759). So, this PR upgrades org.codehaus.janino:commons-compiler to 3.0.0.

## How was this patch tested?
jenkins

Author: Yin Huai <yhuai@databricks.com>

Closes #16025 from yhuai/janino-commons-compile.
2016-11-28 10:09:30 -08:00
Wenchen Fan d31ff9b7ca [SPARK-17732][SQL] Revert ALTER TABLE DROP PARTITION should support comparators
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/15704 will fail if we use int literal in `DROP PARTITION`, and we have reverted it in branch-2.1.

This PR reverts it in master branch, and add a regression test for it, to make sure the master branch is healthy.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16036 from cloud-fan/revert.
2016-11-28 08:46:00 -08:00
Herman van Hovell 38e29824d9 [SPARK-18597][SQL] Do not push-down join conditions to the right side of a LEFT ANTI join
## What changes were proposed in this pull request?
We currently push down join conditions of a Left Anti join to both sides of the join. This is similar to Inner, Left Semi and Existence (a specialized left semi) join. The problem is that this changes the semantics of the join; a left anti join filters out rows that matches the join condition.

This PR fixes this by only pushing down conditions to the left hand side of the join. This is similar to the behavior of left outer join.

## How was this patch tested?
Added tests to `FilterPushdownSuite.scala` and created a SQLQueryTestSuite file for left anti joins with a regression test.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #16026 from hvanhovell/SPARK-18597.
2016-11-28 07:10:52 -08:00
gatorsmile 9f273c5173 [SPARK-17783][SQL] Hide Credentials in CREATE and DESC FORMATTED/EXTENDED a PERSISTENT/TEMP Table for JDBC
### What changes were proposed in this pull request?

We should never expose the Credentials in the EXPLAIN and DESC FORMATTED/EXTENDED command. However, below commands exposed the credentials.

In the related PR: https://github.com/apache/spark/pull/10452

> URL patterns to specify credential seems to be vary between different databases.

Thus, we hide the whole `url` value if it contains the keyword `password`. We also hide the `password` property.

Before the fix, the command outputs look like:

``` SQL
CREATE TABLE tab1
USING org.apache.spark.sql.jdbc
OPTIONS (
 url 'jdbc:h2:mem:testdb0;user=testUser;password=testPass',
 dbtable 'TEST.PEOPLE',
 user 'testUser',
 password '$password')

DESC FORMATTED tab1
DESC EXTENDED tab1
```

Before the fix,
- The output of SQL statement EXPLAIN
```
== Physical Plan ==
ExecutedCommand
   +- CreateDataSourceTableCommand CatalogTable(
	Table: `tab1`
	Created: Wed Nov 16 23:00:10 PST 2016
	Last Access: Wed Dec 31 15:59:59 PST 1969
	Type: MANAGED
	Provider: org.apache.spark.sql.jdbc
	Storage(Properties: [url=jdbc:h2:mem:testdb0;user=testUser;password=testPass, dbtable=TEST.PEOPLE, user=testUser, password=testPass])), false
```

- The output of `DESC FORMATTED`
```
...
|Storage Desc Parameters:    |                                                                  |       |
|  url                       |jdbc:h2:mem:testdb0;user=testUser;password=testPass               |       |
|  dbtable                   |TEST.PEOPLE                                                       |       |
|  user                      |testUser                                                          |       |
|  password                  |testPass                                                          |       |
+----------------------------+------------------------------------------------------------------+-------+
```

- The output of `DESC EXTENDED`
```
|# Detailed Table Information|CatalogTable(
	Table: `default`.`tab1`
	Created: Wed Nov 16 23:00:10 PST 2016
	Last Access: Wed Dec 31 15:59:59 PST 1969
	Type: MANAGED
	Schema: [StructField(NAME,StringType,false), StructField(THEID,IntegerType,false)]
	Provider: org.apache.spark.sql.jdbc
	Storage(Location: file:/Users/xiaoli/IdeaProjects/sparkDelivery/spark-warehouse/tab1, Properties: [url=jdbc:h2:mem:testdb0;user=testUser;password=testPass, dbtable=TEST.PEOPLE, user=testUser, password=testPass]))|       |
```

After the fix,
- The output of SQL statement EXPLAIN
```
== Physical Plan ==
ExecutedCommand
   +- CreateDataSourceTableCommand CatalogTable(
	Table: `tab1`
	Created: Wed Nov 16 22:43:49 PST 2016
	Last Access: Wed Dec 31 15:59:59 PST 1969
	Type: MANAGED
	Provider: org.apache.spark.sql.jdbc
	Storage(Properties: [url=###, dbtable=TEST.PEOPLE, user=testUser, password=###])), false
```
- The output of `DESC FORMATTED`
```
...
|Storage Desc Parameters:    |                                                                  |       |
|  url                       |###                                                               |       |
|  dbtable                   |TEST.PEOPLE                                                       |       |
|  user                      |testUser                                                          |       |
|  password                  |###                                                               |       |
+----------------------------+------------------------------------------------------------------+-------+
```

- The output of `DESC EXTENDED`
```
|# Detailed Table Information|CatalogTable(
	Table: `default`.`tab1`
	Created: Wed Nov 16 22:43:49 PST 2016
	Last Access: Wed Dec 31 15:59:59 PST 1969
	Type: MANAGED
	Schema: [StructField(NAME,StringType,false), StructField(THEID,IntegerType,false)]
	Provider: org.apache.spark.sql.jdbc
	Storage(Location: file:/Users/xiaoli/IdeaProjects/sparkDelivery/spark-warehouse/tab1, Properties: [url=###, dbtable=TEST.PEOPLE, user=testUser, password=###]))|       |
```

### How was this patch tested?

Added test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15358 from gatorsmile/maskCredentials.
2016-11-28 07:04:38 -08:00
Herman van Hovell 70dfdcbbf1 [SPARK-18118][SQL] fix a compilation error due to nested JavaBeans\nRemove this reference. 2016-11-28 04:41:43 -08:00
Kazuaki Ishizaki f075cd9cb7 [SPARK-18118][SQL] fix a compilation error due to nested JavaBeans
## 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 generated java code `SpecificSafeProjection.apply()` for nested JavaBeans is too big. This PR avoids this compilation error by splitting a big code chunk into multiple methods by calling `CodegenContext.splitExpression` at `InitializeJavaBean.doGenCode`
An object reference for JavaBean is stored to an instance variable `javaBean...`. Then, the instance variable will be referenced in the split methods.

Generated code with this PR
````
/* 22098 */   private void apply130_0(InternalRow i) {
...
/* 22125 */     boolean isNull238 = i.isNullAt(2);
/* 22126 */     InternalRow value238 = isNull238 ? null : (i.getStruct(2, 3));
/* 22127 */     boolean isNull236 = false;
/* 22128 */     test.org.apache.spark.sql.JavaDatasetSuite$Nesting1 value236 = null;
/* 22129 */     if (!false && isNull238) {
/* 22130 */
/* 22131 */       final test.org.apache.spark.sql.JavaDatasetSuite$Nesting1 value239 = null;
/* 22132 */       isNull236 = true;
/* 22133 */       value236 = value239;
/* 22134 */     } else {
/* 22135 */
/* 22136 */       final test.org.apache.spark.sql.JavaDatasetSuite$Nesting1 value241 = false ? null : new test.org.apache.spark.sql.JavaDatasetSuite$Nesting1();
/* 22137 */       this.javaBean14 = value241;
/* 22138 */       if (!false) {
/* 22139 */         apply25_0(i);
/* 22140 */         apply25_1(i);
/* 22141 */         apply25_2(i);
/* 22142 */       }
/* 22143 */       isNull236 = false;
/* 22144 */       value236 = value241;
/* 22145 */     }
/* 22146 */     this.javaBean.setField2(value236);
/* 22147 */
/* 22148 */   }
...
/* 22928 */   public java.lang.Object apply(java.lang.Object _i) {
/* 22929 */     InternalRow i = (InternalRow) _i;
/* 22930 */
/* 22931 */     final test.org.apache.spark.sql.JavaDatasetSuite$NestedComplicatedJavaBean value1 = false ? null : new test.org.apache.spark.sql.JavaDatasetSuite$NestedComplicatedJavaBean();
/* 22932 */     this.javaBean = value1;
/* 22933 */     if (!false) {
/* 22934 */       apply130_0(i);
/* 22935 */       apply130_1(i);
/* 22936 */       apply130_2(i);
/* 22937 */       apply130_3(i);
/* 22938 */       apply130_4(i);
/* 22939 */     }
/* 22940 */     if (false) {
/* 22941 */       mutableRow.setNullAt(0);
/* 22942 */     } else {
/* 22943 */
/* 22944 */       mutableRow.update(0, value1);
/* 22945 */     }
/* 22946 */
/* 22947 */     return mutableRow;
/* 22948 */   }
````

## How was this patch tested?

added a test suite into `JavaDatasetSuite.java`

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #16032 from kiszk/SPARK-18118.
2016-11-28 04:18:35 -08:00
Herman van Hovell 454b804991 [SPARK-18604][SQL] Make sure CollapseWindow returns the attributes in the same order.
## What changes were proposed in this pull request?
The `CollapseWindow` optimizer rule changes the order of output attributes. This modifies the output of the plan, which the optimizer cannot do. This also breaks things like `collect()` for which we use a `RowEncoder` that assumes that the output attributes of the executed plan are equal to those outputted by the logical plan.

## How was this patch tested?
I have updated an incorrect test in `CollapseWindowSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #16027 from hvanhovell/SPARK-18604.
2016-11-28 02:56:26 -08:00
Takuya UESHIN 87141622ee [SPARK-18585][SQL] Use ev.isNull = "false" if possible for Janino to have a chance to optimize.
## What changes were proposed in this pull request?

Janino can optimize `true ? a : b` into `a` or `false ? a : b` into `b`, or if/else with literal condition, so we should use literal as `ev.isNull` if possible.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #16008 from ueshin/issues/SPARK-18585.
2016-11-27 23:30:18 -08:00
gatorsmile 07f32c2283 [SPARK-18594][SQL] Name Validation of Databases/Tables
### What changes were proposed in this pull request?
Currently, the name validation checks are limited to table creation. It is enfored by Analyzer rule: `PreWriteCheck`.

However, table renaming and database creation have the same issues. It makes more sense to do the checks in `SessionCatalog`. This PR is to add it into `SessionCatalog`.

### How was this patch tested?
Added test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16018 from gatorsmile/nameValidate.
2016-11-27 19:43:24 -08:00
Dongjoon Hyun 9c03c56460 [SPARK-17251][SQL] Improve OuterReference to be NamedExpression
## What changes were proposed in this pull request?

Currently, `OuterReference` is not `NamedExpression`. So, it raises 'ClassCastException` when it used in projection lists of IN correlated subqueries. This PR aims to support that by making `OuterReference` as `NamedExpression` to show correct error messages.

```scala
scala> sql("CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES 1, 2 AS t1(a)")
scala> sql("CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES 1 AS t2(b)")
scala> sql("SELECT a FROM t1 WHERE a IN (SELECT a FROM t2)").show
java.lang.ClassCastException: org.apache.spark.sql.catalyst.expressions.OuterReference cannot be cast to org.apache.spark.sql.catalyst.expressions.NamedExpression
```

## How was this patch tested?

Pass the Jenkins test with new test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #16015 from dongjoon-hyun/SPARK-17251-2.
2016-11-26 14:57:48 -08:00
Takuya UESHIN a88329d455 [SPARK-18583][SQL] Fix nullability of InputFileName.
## What changes were proposed in this pull request?

The nullability of `InputFileName` should be `false`.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #16007 from ueshin/issues/SPARK-18583.
2016-11-25 20:25:29 -08:00
jiangxingbo e2fb9fd365 [SPARK-18436][SQL] isin causing SQL syntax error with JDBC
## What changes were proposed in this pull request?

The expression `in(empty seq)` is invalid in some data source. Since `in(empty seq)` is always false, we should generate `in(empty seq)` to false literal in optimizer.
The sql `SELECT * FROM t WHERE a IN ()` throws a `ParseException` which is consistent with Hive, don't need to change that behavior.

## How was this patch tested?
Add new test case in `OptimizeInSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15977 from jiangxb1987/isin-empty.
2016-11-25 12:44:34 -08:00
Zhenhua Wang 5ecdc7c5c0 [SPARK-18559][SQL] Fix HLL++ with small relative error
## What changes were proposed in this pull request?

In `HyperLogLogPlusPlus`, if the relative error is so small that p >= 19, it will cause ArrayIndexOutOfBoundsException in `THRESHOLDS(p-4)` . We should check `p` and when p >= 19, regress to the original HLL result and use the small range correction they use.

The pr also fixes the upper bound in the log info in `require()`.
The upper bound is computed by:
```
val relativeSD = 1.106d / Math.pow(Math.E, p * Math.log(2.0d) / 2.0d)
```
which is derived from the equation for computing `p`:
```
val p = 2.0d * Math.log(1.106d / relativeSD) / Math.log(2.0d)
```

## How was this patch tested?

add test cases for:
1. checking validity of parameter relatvieSD
2. estimation with smaller relative error so that p >= 19

Author: Zhenhua Wang <wzh_zju@163.com>
Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #15990 from wzhfy/hllppRsd.
2016-11-25 05:02:48 -08:00
hyukjinkwon 51b1c1551d
[SPARK-3359][BUILD][DOCS] More changes to resolve javadoc 8 errors that will help unidoc/genjavadoc compatibility
## What changes were proposed in this pull request?

This PR only tries to fix things that looks pretty straightforward and were fixed in other previous PRs before.

This PR roughly fixes several things as below:

- Fix unrecognisable class and method links in javadoc by changing it from `[[..]]` to `` `...` ``

  ```
  [error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/DataStreamReader.java:226: error: reference not found
  [error]    * Loads text files and returns a {link DataFrame} whose schema starts with a string column named
  ```

- Fix an exception annotation and remove code backticks in `throws` annotation

  Currently, sbt unidoc with Java 8 complains as below:

  ```
  [error] .../java/org/apache/spark/sql/streaming/StreamingQuery.java:72: error: unexpected text
  [error]    * throws StreamingQueryException, if <code>this</code> query has terminated with an exception.
  ```

  `throws` should specify the correct class name from `StreamingQueryException,` to `StreamingQueryException` without backticks. (see [JDK-8007644](https://bugs.openjdk.java.net/browse/JDK-8007644)).

- Fix `[[http..]]` to `<a href="http..."></a>`.

  ```diff
  -   * [[https://blogs.oracle.com/java-platform-group/entry/diagnosing_tls_ssl_and_https Oracle
  -   * blog page]].
  +   * <a href="https://blogs.oracle.com/java-platform-group/entry/diagnosing_tls_ssl_and_https">
  +   * Oracle blog page</a>.
  ```

   `[[http...]]` link markdown in scaladoc is unrecognisable in javadoc.

- It seems class can't have `return` annotation. So, two cases of this were removed.

  ```
  [error] .../java/org/apache/spark/mllib/regression/IsotonicRegression.java:27: error: invalid use of return
  [error]    * return New instance of IsotonicRegression.
  ```

- Fix < to `&lt;` and > to `&gt;` according to HTML rules.

- Fix `</p>` complaint

- Exclude unrecognisable in javadoc, `constructor`, `todo` and `groupname`.

## How was this patch tested?

Manually tested by `jekyll build` with Java 7 and 8

```
java version "1.7.0_80"
Java(TM) SE Runtime Environment (build 1.7.0_80-b15)
Java HotSpot(TM) 64-Bit Server VM (build 24.80-b11, mixed mode)
```

```
java version "1.8.0_45"
Java(TM) SE Runtime Environment (build 1.8.0_45-b14)
Java HotSpot(TM) 64-Bit Server VM (build 25.45-b02, mixed mode)
```

Note: this does not yet make sbt unidoc suceed with Java 8 yet but it reduces the number of errors with Java 8.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15999 from HyukjinKwon/SPARK-3359-errors.
2016-11-25 11:27:07 +00:00
Nattavut Sutyanyong a367d5ff00 [SPARK-18578][SQL] Full outer join in correlated subquery returns incorrect results
## What changes were proposed in this pull request?

- Raise Analysis exception when correlated predicates exist in the descendant operators of either operand of a Full outer join in a subquery as well as in a FOJ operator itself
- Raise Analysis exception when correlated predicates exists in a Window operator (a side effect inadvertently introduced by SPARK-17348)

## How was this patch tested?

Run sql/test catalyst/test and new test cases, added to SubquerySuite, showing the reported incorrect results.

Author: Nattavut Sutyanyong <nsy.can@gmail.com>

Closes #16005 from nsyca/FOJ-incorrect.1.
2016-11-24 12:07:55 -08:00
Reynold Xin 70ad07a9d2 [SPARK-18522][SQL] Explicit contract for column stats serialization
## What changes were proposed in this pull request?
The current implementation of column stats uses the base64 encoding of the internal UnsafeRow format to persist statistics (in table properties in Hive metastore). This is an internal format that is not stable across different versions of Spark and should NOT be used for persistence. In addition, it would be better if statistics stored in the catalog is human readable.

This pull request introduces the following changes:

1. Created a single ColumnStat class to for all data types. All data types track the same set of statistics.
2. Updated the implementation for stats collection to get rid of the dependency on internal data structures (e.g. InternalRow, or storing DateType as an int32). For example, previously dates were stored as a single integer, but are now stored as java.sql.Date. When we implement the next steps of CBO, we can add code to convert those back into internal types again.
3. Documented clearly what JVM data types are being used to store what data.
4. Defined a simple Map[String, String] interface for serializing and deserializing column stats into/from the catalog.
5. Rearranged the method/function structure so it is more clear what the supported data types are, and also moved how stats are generated into ColumnStat class so they are easy to find.

## How was this patch tested?
Removed most of the original test cases created for column statistics, and added three very simple ones to cover all the cases. The three test cases validate:
1. Roundtrip serialization works.
2. Behavior when analyzing non-existent column or unsupported data type column.
3. Result for stats collection for all valid data types.

Also moved parser related tests into a parser test suite and added an explicit serialization test for the Hive external catalog.

Author: Reynold Xin <rxin@databricks.com>

Closes #15959 from rxin/SPARK-18522.
2016-11-23 20:48:41 +08:00
Wenchen Fan 84284e8c82 [SPARK-18053][SQL] compare unsafe and safe complex-type values correctly
## What changes were proposed in this pull request?

In Spark SQL, some expression may output safe format values, e.g. `CreateArray`, `CreateStruct`, `Cast`, etc. When we compare 2 values, we should be able to compare safe and unsafe formats.

The `GreaterThan`, `LessThan`, etc. in Spark SQL already handles it, but the `EqualTo` doesn't. This PR fixes it.

## How was this patch tested?

new unit test and regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15929 from cloud-fan/type-aware.
2016-11-23 04:15:19 -08:00
hyukjinkwon 2559fb4b40 [SPARK-18179][SQL] Throws analysis exception with a proper message for unsupported argument types in reflect/java_method function
## What changes were proposed in this pull request?

This PR proposes throwing an `AnalysisException` with a proper message rather than `NoSuchElementException` with the message ` key not found: TimestampType` when unsupported types are given to `reflect` and `java_method` functions.

```scala
spark.range(1).selectExpr("reflect('java.lang.String', 'valueOf', cast('1990-01-01' as timestamp))")
```

produces

**Before**

```
java.util.NoSuchElementException: key not found: TimestampType
  at scala.collection.MapLike$class.default(MapLike.scala:228)
  at scala.collection.AbstractMap.default(Map.scala:59)
  at scala.collection.MapLike$class.apply(MapLike.scala:141)
  at scala.collection.AbstractMap.apply(Map.scala:59)
  at org.apache.spark.sql.catalyst.expressions.CallMethodViaReflection$$anonfun$findMethod$1$$anonfun$apply$1.apply(CallMethodViaReflection.scala:159)
...
```

**After**

```
cannot resolve 'reflect('java.lang.String', 'valueOf', CAST('1990-01-01' AS TIMESTAMP))' due to data type mismatch: arguments from the third require boolean, byte, short, integer, long, float, double or string expressions; line 1 pos 0;
'Project [unresolvedalias(reflect(java.lang.String, valueOf, cast(1990-01-01 as timestamp)), Some(<function1>))]
+- Range (0, 1, step=1, splits=Some(2))
...
```

Added message is,

```
arguments from the third require boolean, byte, short, integer, long, float, double or string expressions
```

## How was this patch tested?

Tests added in `CallMethodViaReflection`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15694 from HyukjinKwon/SPARK-18179.
2016-11-22 22:25:27 -08:00
Dilip Biswal 39a1d30636 [SPARK-18533] Raise correct error upon specification of schema for datasource tables created using CTAS
## What changes were proposed in this pull request?
Fixes the inconsistency of error raised between data source and hive serde
tables when schema is specified in CTAS scenario. In the process the grammar for
create table (datasource) is simplified.

**before:**
``` SQL
spark-sql> create table t2 (c1 int, c2 int) using parquet as select * from t1;
Error in query:
mismatched input 'as' expecting {<EOF>, '.', 'OPTIONS', 'CLUSTERED', 'PARTITIONED'}(line 1, pos 64)

== SQL ==
create table t2 (c1 int, c2 int) using parquet as select * from t1
----------------------------------------------------------------^^^
```

**After:**
```SQL
spark-sql> create table t2 (c1 int, c2 int) using parquet as select * from t1
         > ;
Error in query:
Operation not allowed: Schema may not be specified in a Create Table As Select (CTAS) statement(line 1, pos 0)

== SQL ==
create table t2 (c1 int, c2 int) using parquet as select * from t1
^^^
```
## How was this patch tested?
Added a new test in CreateTableAsSelectSuite

Author: Dilip Biswal <dbiswal@us.ibm.com>

Closes #15968 from dilipbiswal/ctas.
2016-11-22 15:57:07 -08:00
Burak Yavuz bdc8153e86 [SPARK-18465] Add 'IF EXISTS' clause to 'UNCACHE' to not throw exceptions when table doesn't exist
## What changes were proposed in this pull request?

While this behavior is debatable, consider the following use case:
```sql
UNCACHE TABLE foo;
CACHE TABLE foo AS
SELECT * FROM bar
```
The command above fails the first time you run it. But I want to run the command above over and over again, and I don't want to change my code just for the first run of it.
The issue is that subsequent `CACHE TABLE` commands do not overwrite the existing table.

Now we can do:
```sql
UNCACHE TABLE IF EXISTS foo;
CACHE TABLE foo AS
SELECT * FROM bar
```

## How was this patch tested?

Unit tests

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #15896 from brkyvz/uncache.
2016-11-22 13:03:50 -08:00
Nattavut Sutyanyong 45ea46b7b3 [SPARK-18504][SQL] Scalar subquery with extra group by columns returning incorrect result
## What changes were proposed in this pull request?

This PR blocks an incorrect result scenario in scalar subquery where there are GROUP BY column(s)
that are not part of the correlated predicate(s).

Example:
// Incorrect result
Seq(1).toDF("c1").createOrReplaceTempView("t1")
Seq((1,1),(1,2)).toDF("c1","c2").createOrReplaceTempView("t2")
sql("select (select sum(-1) from t2 where t1.c1=t2.c1 group by t2.c2) from t1").show

// How can selecting a scalar subquery from a 1-row table return 2 rows?

## How was this patch tested?
sql/test, catalyst/test
new test case covering the reported problem is added to SubquerySuite.scala

Author: Nattavut Sutyanyong <nsy.can@gmail.com>

Closes #15936 from nsyca/scalarSubqueryIncorrect-1.
2016-11-22 12:06:21 -08:00
Wenchen Fan bb152cdfbb [SPARK-18519][SQL] map type can not be used in EqualTo
## What changes were proposed in this pull request?

Technically map type is not orderable, but can be used in equality comparison. However, due to the limitation of the current implementation, map type can't be used in equality comparison so that it can't be join key or grouping key.

This PR makes this limitation explicit, to avoid wrong result.

## How was this patch tested?

updated tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15956 from cloud-fan/map-type.
2016-11-22 09:16:20 -08:00
Takuya UESHIN 9f262ae163 [SPARK-18398][SQL] Fix nullabilities of MapObjects and ExternalMapToCatalyst.
## What changes were proposed in this pull request?

The nullabilities of `MapObject` can be made more strict by relying on `inputObject.nullable` and `lambdaFunction.nullable`.

Also `ExternalMapToCatalyst.dataType` can be made more strict by relying on `valueConverter.nullable`.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #15840 from ueshin/issues/SPARK-18398.
2016-11-21 05:50:35 -08:00
Takuya UESHIN 6585479749 [SPARK-18467][SQL] Extracts method for preparing arguments from StaticInvoke, Invoke and NewInstance and modify to short circuit if arguments have null when needNullCheck == true.
## What changes were proposed in this pull request?

This pr extracts method for preparing arguments from `StaticInvoke`, `Invoke` and `NewInstance` and modify to short circuit if arguments have `null` when `propageteNull == true`.

The steps are as follows:

1. Introduce `InvokeLike` to extract common logic from `StaticInvoke`, `Invoke` and `NewInstance` to prepare arguments.
`StaticInvoke` and `Invoke` had a risk to exceed 64kb JVM limit to prepare arguments but after this patch they can handle them because they share the preparing code of NewInstance, which handles the limit well.

2. Remove unneeded null checking and fix nullability of `NewInstance`.
Avoid some of nullabilty checking which are not needed because the expression is not nullable.

3. Modify to short circuit if arguments have `null` when `needNullCheck == true`.
If `needNullCheck == true`, preparing arguments can be skipped if we found one of them is `null`, so modified to short circuit in the case.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #15901 from ueshin/issues/SPARK-18467.
2016-11-21 12:05:01 +08:00
Herman van Hovell 7ca7a63524 [SPARK-15214][SQL] Code-generation for Generate
## What changes were proposed in this pull request?

This PR adds code generation to `Generate`. It supports two code paths:
- General `TraversableOnce` based iteration. This used for regular `Generator` (code generation supporting) expressions. This code path expects the expression to return a `TraversableOnce[InternalRow]` and it will iterate over the returned collection. This PR adds code generation for the `stack` generator.
- Specialized `ArrayData/MapData` based iteration. This is used for the `explode`, `posexplode` & `inline` functions and operates directly on the `ArrayData`/`MapData` result that the child of the generator returns.

### Benchmarks
I have added some benchmarks and it seems we can create a nice speedup for explode:
#### Environment
```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6
Intel(R) Core(TM) i7-4980HQ CPU  2.80GHz
```
#### Explode Array
##### Before
```
generate explode array:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate explode array wholestage off         7377 / 7607          2.3         439.7       1.0X
generate explode array wholestage on          6055 / 6086          2.8         360.9       1.2X
```
##### After
```
generate explode array:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate explode array wholestage off         7432 / 7696          2.3         443.0       1.0X
generate explode array wholestage on           631 /  646         26.6          37.6      11.8X
```
#### Explode Map
##### Before
```
generate explode map:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate explode map wholestage off         12792 / 12848          1.3         762.5       1.0X
generate explode map wholestage on          11181 / 11237          1.5         666.5       1.1X
```
##### After
```
generate explode map:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate explode map wholestage off         10949 / 10972          1.5         652.6       1.0X
generate explode map wholestage on             870 /  913         19.3          51.9      12.6X
```
#### Posexplode
##### Before
```
generate posexplode array:               Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate posexplode array wholestage off      7547 / 7580          2.2         449.8       1.0X
generate posexplode array wholestage on       5786 / 5838          2.9         344.9       1.3X
```
##### After
```
generate posexplode array:               Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate posexplode array wholestage off      7535 / 7548          2.2         449.1       1.0X
generate posexplode array wholestage on        620 /  624         27.1          37.0      12.1X
```
#### Inline
##### Before
```
generate inline array:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate inline array wholestage off          6935 / 6978          2.4         413.3       1.0X
generate inline array wholestage on           6360 / 6400          2.6         379.1       1.1X
```
##### After
```
generate inline array:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate inline array wholestage off          6940 / 6966          2.4         413.6       1.0X
generate inline array wholestage on           1002 / 1012         16.7          59.7       6.9X
```
#### Stack
##### Before
```
generate stack:                          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate stack wholestage off               12980 / 13104          1.3         773.7       1.0X
generate stack wholestage on                11566 / 11580          1.5         689.4       1.1X
```
##### After
```
generate stack:                          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate stack wholestage off               12875 / 12949          1.3         767.4       1.0X
generate stack wholestage on                   840 /  845         20.0          50.0      15.3X
```
## How was this patch tested?

Existing tests.

Author: Herman van Hovell <hvanhovell@databricks.com>
Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #13065 from hvanhovell/SPARK-15214.
2016-11-19 23:55:09 -08:00
Reynold Xin a64f25d8b4 [SQL] Fix documentation for Concat and ConcatWs 2016-11-19 21:57:49 -08:00
Reynold Xin bce9a03677 [SPARK-18508][SQL] Fix documentation error for DateDiff
## What changes were proposed in this pull request?
The previous documentation and example for DateDiff was wrong.

## How was this patch tested?
Doc only change.

Author: Reynold Xin <rxin@databricks.com>

Closes #15937 from rxin/datediff-doc.
2016-11-19 21:57:09 -08:00
hyukjinkwon d5b1d5fc80
[SPARK-18445][BUILD][DOCS] Fix the markdown for Note:/NOTE:/Note that/'''Note:''' across Scala/Java API documentation
## What changes were proposed in this pull request?

It seems in Scala/Java,

- `Note:`
- `NOTE:`
- `Note that`
- `'''Note:'''`
- `note`

This PR proposes to fix those to `note` to be consistent.

**Before**

- Scala
  ![2016-11-17 6 16 39](https://cloud.githubusercontent.com/assets/6477701/20383180/1a7aed8c-acf2-11e6-9611-5eaf6d52c2e0.png)

- Java
  ![2016-11-17 6 14 41](https://cloud.githubusercontent.com/assets/6477701/20383096/c8ffc680-acf1-11e6-914a-33460bf1401d.png)

**After**

- Scala
  ![2016-11-17 6 16 44](https://cloud.githubusercontent.com/assets/6477701/20383167/09940490-acf2-11e6-937a-0d5e1dc2cadf.png)

- Java
  ![2016-11-17 6 13 39](https://cloud.githubusercontent.com/assets/6477701/20383132/e7c2a57e-acf1-11e6-9c47-b849674d4d88.png)

## How was this patch tested?

The notes were found via

```bash
grep -r "NOTE: " . | \ # Note:|NOTE:|Note that|'''Note:'''
grep -v "// NOTE: " | \  # starting with // does not appear in API documentation.
grep -E '.scala|.java' | \ # java/scala files
grep -v Suite | \ # exclude tests
grep -v Test | \ # exclude tests
grep -e 'org.apache.spark.api.java' \ # packages appear in API documenation
-e 'org.apache.spark.api.java.function' \ # note that this is a regular expression. So actual matches were mostly `org/apache/spark/api/java/functions ...`
-e 'org.apache.spark.api.r' \
...
```

```bash
grep -r "Note that " . | \ # Note:|NOTE:|Note that|'''Note:'''
grep -v "// Note that " | \  # starting with // does not appear in API documentation.
grep -E '.scala|.java' | \ # java/scala files
grep -v Suite | \ # exclude tests
grep -v Test | \ # exclude tests
grep -e 'org.apache.spark.api.java' \ # packages appear in API documenation
-e 'org.apache.spark.api.java.function' \
-e 'org.apache.spark.api.r' \
...
```

```bash
grep -r "Note: " . | \ # Note:|NOTE:|Note that|'''Note:'''
grep -v "// Note: " | \  # starting with // does not appear in API documentation.
grep -E '.scala|.java' | \ # java/scala files
grep -v Suite | \ # exclude tests
grep -v Test | \ # exclude tests
grep -e 'org.apache.spark.api.java' \ # packages appear in API documenation
-e 'org.apache.spark.api.java.function' \
-e 'org.apache.spark.api.r' \
...
```

```bash
grep -r "'''Note:'''" . | \ # Note:|NOTE:|Note that|'''Note:'''
grep -v "// '''Note:''' " | \  # starting with // does not appear in API documentation.
grep -E '.scala|.java' | \ # java/scala files
grep -v Suite | \ # exclude tests
grep -v Test | \ # exclude tests
grep -e 'org.apache.spark.api.java' \ # packages appear in API documenation
-e 'org.apache.spark.api.java.function' \
-e 'org.apache.spark.api.r' \
...
```

And then fixed one by one comparing with API documentation/access modifiers.

After that, manually tested via `jekyll build`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15889 from HyukjinKwon/SPARK-18437.
2016-11-19 11:24:15 +00:00
Takuya UESHIN 170eeb345f [SPARK-18442][SQL] Fix nullability of WrapOption.
## What changes were proposed in this pull request?

The nullability of `WrapOption` should be `false`.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #15887 from ueshin/issues/SPARK-18442.
2016-11-17 11:21:08 +08:00
gatorsmile 608ecc512b [SPARK-18415][SQL] Weird Plan Output when CTE used in RunnableCommand
### What changes were proposed in this pull request?
Currently, when CTE is used in RunnableCommand, the Analyzer does not replace the logical node `With`. The child plan of RunnableCommand is not resolved. Thus, the output of the `With` plan node looks very confusing.
For example,
```
sql(
  """
    |CREATE VIEW cte_view AS
    |WITH w AS (SELECT 1 AS n), cte1 (select 2), cte2 as (select 3)
    |SELECT n FROM w
  """.stripMargin).explain()
```
The output is like
```
ExecutedCommand
   +- CreateViewCommand `cte_view`, WITH w AS (SELECT 1 AS n), cte1 (select 2), cte2 as (select 3)
SELECT n FROM w, false, false, PersistedView
         +- 'With [(w,SubqueryAlias w
+- Project [1 AS n#16]
   +- OneRowRelation$
), (cte1,'SubqueryAlias cte1
+- 'Project [unresolvedalias(2, None)]
   +- OneRowRelation$
), (cte2,'SubqueryAlias cte2
+- 'Project [unresolvedalias(3, None)]
   +- OneRowRelation$
)]
            +- 'Project ['n]
               +- 'UnresolvedRelation `w`
```
After the fix, the output is as shown below.
```
ExecutedCommand
   +- CreateViewCommand `cte_view`, WITH w AS (SELECT 1 AS n), cte1 (select 2), cte2 as (select 3)
SELECT n FROM w, false, false, PersistedView
         +- CTE [w, cte1, cte2]
            :  :- SubqueryAlias w
            :  :  +- Project [1 AS n#16]
            :  :     +- OneRowRelation$
            :  :- 'SubqueryAlias cte1
            :  :  +- 'Project [unresolvedalias(2, None)]
            :  :     +- OneRowRelation$
            :  +- 'SubqueryAlias cte2
            :     +- 'Project [unresolvedalias(3, None)]
            :        +- OneRowRelation$
            +- 'Project ['n]
               +- 'UnresolvedRelation `w`
```

BTW, this PR also fixes the output of the view type.

### How was this patch tested?
Manual

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15854 from gatorsmile/cteName.
2016-11-16 08:25:15 -08:00
Xianyang Liu 7569cf6cb8
[SPARK-18420][BUILD] Fix the errors caused by lint check in Java
## What changes were proposed in this pull request?

Small fix, fix the errors caused by lint check in Java

- Clear unused objects and `UnusedImports`.
- Add comments around the method `finalize` of `NioBufferedFileInputStream`to turn off checkstyle.
- Cut the line which is longer than 100 characters into two lines.

## How was this patch tested?
Travis CI.
```
$ build/mvn -T 4 -q -DskipTests -Pyarn -Phadoop-2.3 -Pkinesis-asl -Phive -Phive-thriftserver install
$ dev/lint-java
```
Before:
```
Checkstyle checks failed at following occurrences:
[ERROR] src/main/java/org/apache/spark/network/util/TransportConf.java:[21,8] (imports) UnusedImports: Unused import - org.apache.commons.crypto.cipher.CryptoCipherFactory.
[ERROR] src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java:[516,5] (modifier) RedundantModifier: Redundant 'public' modifier.
[ERROR] src/main/java/org/apache/spark/io/NioBufferedFileInputStream.java:[133] (coding) NoFinalizer: Avoid using finalizer method.
[ERROR] src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeMapData.java:[71] (sizes) LineLength: Line is longer than 100 characters (found 113).
[ERROR] src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java:[112] (sizes) LineLength: Line is longer than 100 characters (found 110).
[ERROR] src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java:[31,17] (modifier) ModifierOrder: 'static' modifier out of order with the JLS suggestions.
[ERROR]src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionWithElasticNetExample.java:[64] (sizes) LineLength: Line is longer than 100 characters (found 103).
[ERROR] src/main/java/org/apache/spark/examples/ml/JavaInteractionExample.java:[22,8] (imports) UnusedImports: Unused import - org.apache.spark.ml.linalg.Vectors.
[ERROR] src/main/java/org/apache/spark/examples/ml/JavaInteractionExample.java:[51] (regexp) RegexpSingleline: No trailing whitespace allowed.
```

After:
```
$ build/mvn -T 4 -q -DskipTests -Pyarn -Phadoop-2.3 -Pkinesis-asl -Phive -Phive-thriftserver install
$ dev/lint-java
Using `mvn` from path: /home/travis/build/ConeyLiu/spark/build/apache-maven-3.3.9/bin/mvn
Checkstyle checks passed.
```

Author: Xianyang Liu <xyliu0530@icloud.com>

Closes #15865 from ConeyLiu/master.
2016-11-16 11:59:00 +00:00
Dongjoon Hyun 74f5c2176d [SPARK-18433][SQL] Improve DataSource option keys to be more case-insensitive
## What changes were proposed in this pull request?

This PR aims to improve DataSource option keys to be more case-insensitive

DataSource partially use CaseInsensitiveMap in code-path. For example, the following fails to find url.

```scala
val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2)
df.write.format("jdbc")
    .option("UrL", url1)
    .option("dbtable", "TEST.SAVETEST")
    .options(properties.asScala)
    .save()
```

This PR makes DataSource options to use CaseInsensitiveMap internally and also makes DataSource to use CaseInsensitiveMap generally except `InMemoryFileIndex` and `InsertIntoHadoopFsRelationCommand`. We can not pass them CaseInsensitiveMap because they creates new case-sensitive HadoopConfs by calling newHadoopConfWithOptions(options) inside.

## How was this patch tested?

Pass the Jenkins test with newly added test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15884 from dongjoon-hyun/SPARK-18433.
2016-11-16 17:12:18 +08:00
Wenchen Fan 4ac9759f80 [SPARK-18377][SQL] warehouse path should be a static conf
## What changes were proposed in this pull request?

it's weird that every session can set its own warehouse path at runtime, we should forbid it and make it a static conf.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15825 from cloud-fan/warehouse.
2016-11-15 20:24:36 -08:00
Herman van Hovell 4b35d13bac [SPARK-18300][SQL] Fix scala 2.10 build for FoldablePropagation
## What changes were proposed in this pull request?
Commit f14ae4900a broke the scala 2.10 build. This PR fixes this by simplifying the used pattern match.

## How was this patch tested?
Tested building manually. Ran `build/sbt -Dscala-2.10 -Pscala-2.10 package`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #15891 from hvanhovell/SPARK-18300-scala-2.10.
2016-11-15 16:55:02 -08:00
Dongjoon Hyun 3ce057d001 [SPARK-17732][SQL] ALTER TABLE DROP PARTITION should support comparators
## What changes were proposed in this pull request?

This PR aims to support `comparators`, e.g. '<', '<=', '>', '>=', again in Apache Spark 2.0 for backward compatibility.

**Spark 1.6**

``` scala
scala> sql("CREATE TABLE sales(id INT) PARTITIONED BY (country STRING, quarter STRING)")
res0: org.apache.spark.sql.DataFrame = [result: string]

scala> sql("ALTER TABLE sales DROP PARTITION (country < 'KR')")
res1: org.apache.spark.sql.DataFrame = [result: string]
```

**Spark 2.0**

``` scala
scala> sql("CREATE TABLE sales(id INT) PARTITIONED BY (country STRING, quarter STRING)")
res0: org.apache.spark.sql.DataFrame = []

scala> sql("ALTER TABLE sales DROP PARTITION (country < 'KR')")
org.apache.spark.sql.catalyst.parser.ParseException:
mismatched input '<' expecting {')', ','}(line 1, pos 42)
```

After this PR, it's supported.

## How was this patch tested?

Pass the Jenkins test with a newly added testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15704 from dongjoon-hyun/SPARK-17732-2.
2016-11-15 15:59:04 -08:00
Herman van Hovell f14ae4900a [SPARK-18300][SQL] Do not apply foldable propagation with expand as a child.
## What changes were proposed in this pull request?
The `FoldablePropagation` optimizer rule, pulls foldable values out from under an `Expand`. This breaks the `Expand` in two ways:

- It rewrites the output attributes of the `Expand`. We explicitly define output attributes for `Expand`, these are (unfortunately) considered as part of the expressions of the `Expand` and can be rewritten.
- Expand can actually change the column (it will typically re-use the attributes or the underlying plan). This means that we cannot safely propagate the expressions from under an `Expand`.

This PR fixes this and (hopefully) other issues by explicitly whitelisting allowed operators.

## How was this patch tested?
Added tests to `FoldablePropagationSuite` and to `SQLQueryTestSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #15857 from hvanhovell/SPARK-18300.
2016-11-15 06:59:25 -08:00
gatorsmile 86430cc4e8 [SPARK-18430][SQL] Fixed Exception Messages when Hitting an Invocation Exception of Function Lookup
### What changes were proposed in this pull request?
When the exception is an invocation exception during function lookup, we return a useless/confusing error message:

For example,
```Scala
df.selectExpr("concat_ws()")
```
Below is the error message we got:
```
null; line 1 pos 0
org.apache.spark.sql.AnalysisException: null; line 1 pos 0
```

To get the meaningful error message, we need to get the cause. The fix is exactly the same as what we did in https://github.com/apache/spark/pull/12136. After the fix, the message we got is the exception issued in the constuctor of function implementation:
```
requirement failed: concat_ws requires at least one argument.; line 1 pos 0
org.apache.spark.sql.AnalysisException: requirement failed: concat_ws requires at least one argument.; line 1 pos 0
```

### How was this patch tested?
Added test cases.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15878 from gatorsmile/functionNotFound.
2016-11-14 21:21:34 -08:00
Michael Armbrust c07187823a [SPARK-18124] Observed delay based Event Time Watermarks
This PR adds a new method `withWatermark` to the `Dataset` API, which can be used specify an _event time watermark_.  An event time watermark allows the streaming engine to reason about the point in time after which we no longer expect to see late data.  This PR also has augmented `StreamExecution` to use this watermark for several purposes:
  - To know when a given time window aggregation is finalized and thus results can be emitted when using output modes that do not allow updates (e.g. `Append` mode).
  - To minimize the amount of state that we need to keep for on-going aggregations, by evicting state for groups that are no longer expected to change.  Although, we do still maintain all state if the query requires (i.e. if the event time is not present in the `groupBy` or when running in `Complete` mode).

An example that emits windowed counts of records, waiting up to 5 minutes for late data to arrive.
```scala
df.withWatermark("eventTime", "5 minutes")
  .groupBy(window($"eventTime", "1 minute") as 'window)
  .count()
  .writeStream
  .format("console")
  .mode("append") // In append mode, we only output finalized aggregations.
  .start()
```

### Calculating the watermark.
The current event time is computed by looking at the `MAX(eventTime)` seen this epoch across all of the partitions in the query minus some user defined _delayThreshold_.  An additional constraint is that the watermark must increase monotonically.

Note that since we must coordinate this value across partitions occasionally, the actual watermark used is only guaranteed to be at least `delay` behind the actual event time.  In some cases we may still process records that arrive more than delay late.

This mechanism was chosen for the initial implementation over processing time for two reasons:
  - it is robust to downtime that could affect processing delay
  - it does not require syncing of time or timezones between the producer and the processing engine.

### Other notable implementation details
 - A new trigger metric `eventTimeWatermark` outputs the current value of the watermark.
 - We mark the event time column in the `Attribute` metadata using the key `spark.watermarkDelay`.  This allows downstream operations to know which column holds the event time.  Operations like `window` propagate this metadata.
 - `explain()` marks the watermark with a suffix of `-T${delayMs}` to ease debugging of how this information is propagated.
 - Currently, we don't filter out late records, but instead rely on the state store to avoid emitting records that are both added and filtered in the same epoch.

### Remaining in this PR
 - [ ] The test for recovery is currently failing as we don't record the watermark used in the offset log.  We will need to do so to ensure determinism, but this is deferred until #15626 is merged.

### Other follow-ups
There are some natural additional features that we should consider for future work:
 - Ability to write records that arrive too late to some external store in case any out-of-band remediation is required.
 - `Update` mode so you can get partial results before a group is evicted.
 - Other mechanisms for calculating the watermark.  In particular a watermark based on quantiles would be more robust to outliers.

Author: Michael Armbrust <michael@databricks.com>

Closes #15702 from marmbrus/watermarks.
2016-11-14 16:46:26 -08:00
Nattavut Sutyanyong bd85603ba5 [SPARK-17348][SQL] Incorrect results from subquery transformation
## What changes were proposed in this pull request?

Return an Analysis exception when there is a correlated non-equality predicate in a subquery and the correlated column from the outer reference is not from the immediate parent operator of the subquery. This PR prevents incorrect results from subquery transformation in such case.

Test cases, both positive and negative tests, are added.

## How was this patch tested?

sql/test, catalyst/test, hive/test, and scenarios that will produce incorrect results without this PR and product correct results when subquery transformation does happen.

Author: Nattavut Sutyanyong <nsy.can@gmail.com>

Closes #15763 from nsyca/spark-17348.
2016-11-14 20:59:15 +01:00
Ryan Blue 6e95325fc3 [SPARK-18387][SQL] Add serialization to checkEvaluation.
## What changes were proposed in this pull request?

This removes the serialization test from RegexpExpressionsSuite and
replaces it by serializing all expressions in checkEvaluation.

This also fixes math constant expressions by making LeafMathExpression
Serializable and fixes NumberFormat values that are null or invalid
after serialization.

## How was this patch tested?

This patch is to tests.

Author: Ryan Blue <blue@apache.org>

Closes #15847 from rdblue/SPARK-18387-fix-serializable-expressions.
2016-11-11 13:52:10 -08:00
Eric Liang a3356343cb [SPARK-18185] Fix all forms of INSERT / OVERWRITE TABLE for Datasource tables
## What changes were proposed in this pull request?

As of current 2.1, INSERT OVERWRITE with dynamic partitions against a Datasource table will overwrite the entire table instead of only the partitions matching the static keys, as in Hive. It also doesn't respect custom partition locations.

This PR adds support for all these operations to Datasource tables managed by the Hive metastore. It is implemented as follows
- During planning time, the full set of partitions affected by an INSERT or OVERWRITE command is read from the Hive metastore.
- The planner identifies any partitions with custom locations and includes this in the write task metadata.
- FileFormatWriter tasks refer to this custom locations map when determining where to write for dynamic partition output.
- When the write job finishes, the set of written partitions is compared against the initial set of matched partitions, and the Hive metastore is updated to reflect the newly added / removed partitions.

It was necessary to introduce a method for staging files with absolute output paths to `FileCommitProtocol`. These files are not handled by the Hadoop output committer but are moved to their final locations when the job commits.

The overwrite behavior of legacy Datasource tables is also changed: no longer will the entire table be overwritten if a partial partition spec is present.

cc cloud-fan yhuai

## How was this patch tested?

Unit tests, existing tests.

Author: Eric Liang <ekl@databricks.com>
Author: Wenchen Fan <wenchen@databricks.com>

Closes #15814 from ericl/sc-5027.
2016-11-10 17:00:43 -08:00
Wenchen Fan 2f7461f313 [SPARK-17990][SPARK-18302][SQL] correct several partition related behaviours of ExternalCatalog
## What changes were proposed in this pull request?

This PR corrects several partition related behaviors of `ExternalCatalog`:

1. default partition location should not always lower case the partition column names in path string(fix `HiveExternalCatalog`)
2. rename partition should not always lower case the partition column names in updated partition path string(fix `HiveExternalCatalog`)
3. rename partition should update the partition location only for managed table(fix `InMemoryCatalog`)
4. create partition with existing directory should be fine(fix `InMemoryCatalog`)
5. create partition with non-existing directory should create that directory(fix `InMemoryCatalog`)
6. drop partition from external table should not delete the directory(fix `InMemoryCatalog`)

## How was this patch tested?

new tests in `ExternalCatalogSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15797 from cloud-fan/partition.
2016-11-10 13:42:48 -08:00
Ryan Blue d4028de976 [SPARK-18368][SQL] Fix regexp replace when serialized
## What changes were proposed in this pull request?

This makes the result value both transient and lazy, so that if the RegExpReplace object is initialized then serialized, `result: StringBuffer` will be correctly initialized.

## How was this patch tested?

* Verified that this patch fixed the query that found the bug.
* Added a test case that fails without the fix.

Author: Ryan Blue <blue@apache.org>

Closes #15834 from rdblue/SPARK-18368-fix-regexp-replace.
2016-11-09 11:00:53 -08:00
Yin Huai 47636618a5 Revert "[SPARK-18368] Fix regexp_replace with task serialization."
This reverts commit b9192bb3ff.
2016-11-09 10:47:29 -08:00
Ryan Blue b9192bb3ff [SPARK-18368] Fix regexp_replace with task serialization.
## What changes were proposed in this pull request?

This makes the result value both transient and lazy, so that if the RegExpReplace object is initialized then serialized, `result: StringBuffer` will be correctly initialized.

## How was this patch tested?

* Verified that this patch fixed the query that found the bug.
* Added a test case that fails without the fix.

Author: Ryan Blue <blue@apache.org>

Closes #15816 from rdblue/SPARK-18368-fix-regexp-replace.
2016-11-08 23:47:48 -08:00
jiangxingbo 344dcad701 [SPARK-17868][SQL] Do not use bitmasks during parsing and analysis of CUBE/ROLLUP/GROUPING SETS
## What changes were proposed in this pull request?

We generate bitmasks for grouping sets during the parsing process, and use these during analysis. These bitmasks are difficult to work with in practice and have lead to numerous bugs. This PR removes these and use actual sets instead, however we still need to generate these offsets for the grouping_id.

This PR does the following works:
1. Replace bitmasks by actual grouping sets durning Parsing/Analysis stage of CUBE/ROLLUP/GROUPING SETS;
2. Add new testsuite `ResolveGroupingAnalyticsSuite` to test the `Analyzer.ResolveGroupingAnalytics` rule directly;
3. Fix a minor bug in `ResolveGroupingAnalytics`.
## How was this patch tested?

By existing test cases, and add new testsuite `ResolveGroupingAnalyticsSuite` to test directly.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15484 from jiangxb1987/group-set.
2016-11-08 15:11:03 +01:00
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
hyukjinkwon 3eda05703f [SPARK-18295][SQL] Make to_json function null safe (matching it to from_json)
## What changes were proposed in this pull request?

This PR proposes to match up the behaviour of `to_json` to `from_json` function for null-safety.

Currently, it throws `NullPointException` but this PR fixes this to produce `null` instead.

with the data below:

```scala
import spark.implicits._

val df = Seq(Some(Tuple1(Tuple1(1))), None).toDF("a")
df.show()
```

```
+----+
|   a|
+----+
| [1]|
|null|
+----+
```

the codes below

```scala
import org.apache.spark.sql.functions._

df.select(to_json($"a")).show()
```

produces..

**Before**

throws `NullPointException` as below:

```
java.lang.NullPointerException
  at org.apache.spark.sql.catalyst.json.JacksonGenerator.org$apache$spark$sql$catalyst$json$JacksonGenerator$$writeFields(JacksonGenerator.scala:138)
  at org.apache.spark.sql.catalyst.json.JacksonGenerator$$anonfun$write$1.apply$mcV$sp(JacksonGenerator.scala:194)
  at org.apache.spark.sql.catalyst.json.JacksonGenerator.org$apache$spark$sql$catalyst$json$JacksonGenerator$$writeObject(JacksonGenerator.scala:131)
  at org.apache.spark.sql.catalyst.json.JacksonGenerator.write(JacksonGenerator.scala:193)
  at org.apache.spark.sql.catalyst.expressions.StructToJson.eval(jsonExpressions.scala:544)
  at org.apache.spark.sql.catalyst.expressions.Alias.eval(namedExpressions.scala:142)
  at org.apache.spark.sql.catalyst.expressions.InterpretedProjection.apply(Projection.scala:48)
  at org.apache.spark.sql.catalyst.expressions.InterpretedProjection.apply(Projection.scala:30)
  at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
```

**After**

```
+---------------+
|structtojson(a)|
+---------------+
|       {"_1":1}|
|           null|
+---------------+
```

## How was this patch tested?

Unit test in `JsonExpressionsSuite.scala` and `JsonFunctionsSuite.scala`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15792 from HyukjinKwon/SPARK-18295.
2016-11-07 16:54:40 -08:00
Kazuaki Ishizaki 19cf208063 [SPARK-17490][SQL] Optimize SerializeFromObject() for a primitive array
## What changes were proposed in this pull request?

Waiting for merging #13680

This PR optimizes `SerializeFromObject()` for an primitive array. This is derived from #13758 to address one of problems by using a simple way in #13758.

The current implementation always generates `GenericArrayData` from `SerializeFromObject()` for any type of an array in a logical plan. This involves a boxing at a constructor of `GenericArrayData` when `SerializedFromObject()` has an primitive array.

This PR enables to generate `UnsafeArrayData` from `SerializeFromObject()` for a primitive array. It can avoid boxing to create an instance of `ArrayData` in the generated code by Catalyst.

This PR also generate `UnsafeArrayData` in a case for `RowEncoder.serializeFor` or `CatalystTypeConverters.createToCatalystConverter`.

Performance improvement of `SerializeFromObject()` is up to 2.0x

```
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)

Without this PR
Write an array in Dataset:               Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            556 /  608         15.1          66.3       1.0X
Double                                        1668 / 1746          5.0         198.8       0.3X

with this PR
Write an array in Dataset:               Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            352 /  401         23.8          42.0       1.0X
Double                                         821 /  885         10.2          97.9       0.4X
```

Here is an example program that will happen in mllib as described in [SPARK-16070](https://issues.apache.org/jira/browse/SPARK-16070).

```
sparkContext.parallelize(Seq(Array(1, 2)), 1).toDS.map(e => e).show
```

Generated code before applying this PR

``` java
/* 039 */   protected void processNext() throws java.io.IOException {
/* 040 */     while (inputadapter_input.hasNext()) {
/* 041 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 042 */       int[] inputadapter_value = (int[])inputadapter_row.get(0, null);
/* 043 */
/* 044 */       Object mapelements_obj = ((Expression) references[0]).eval(null);
/* 045 */       scala.Function1 mapelements_value1 = (scala.Function1) mapelements_obj;
/* 046 */
/* 047 */       boolean mapelements_isNull = false || false;
/* 048 */       int[] mapelements_value = null;
/* 049 */       if (!mapelements_isNull) {
/* 050 */         Object mapelements_funcResult = null;
/* 051 */         mapelements_funcResult = mapelements_value1.apply(inputadapter_value);
/* 052 */         if (mapelements_funcResult == null) {
/* 053 */           mapelements_isNull = true;
/* 054 */         } else {
/* 055 */           mapelements_value = (int[]) mapelements_funcResult;
/* 056 */         }
/* 057 */
/* 058 */       }
/* 059 */       mapelements_isNull = mapelements_value == null;
/* 060 */
/* 061 */       serializefromobject_argIsNulls[0] = mapelements_isNull;
/* 062 */       serializefromobject_argValue = mapelements_value;
/* 063 */
/* 064 */       boolean serializefromobject_isNull = false;
/* 065 */       for (int idx = 0; idx < 1; idx++) {
/* 066 */         if (serializefromobject_argIsNulls[idx]) { serializefromobject_isNull = true; break; }
/* 067 */       }
/* 068 */
/* 069 */       final ArrayData serializefromobject_value = serializefromobject_isNull ? null : new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_argValue);
/* 070 */       serializefromobject_holder.reset();
/* 071 */
/* 072 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 073 */
/* 074 */       if (serializefromobject_isNull) {
/* 075 */         serializefromobject_rowWriter.setNullAt(0);
/* 076 */       } else {
/* 077 */         // Remember the current cursor so that we can calculate how many bytes are
/* 078 */         // written later.
/* 079 */         final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
/* 080 */
/* 081 */         if (serializefromobject_value instanceof UnsafeArrayData) {
/* 082 */           final int serializefromobject_sizeInBytes = ((UnsafeArrayData) serializefromobject_value).getSizeInBytes();
/* 083 */           // grow the global buffer before writing data.
/* 084 */           serializefromobject_holder.grow(serializefromobject_sizeInBytes);
/* 085 */           ((UnsafeArrayData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 086 */           serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
/* 087 */
/* 088 */         } else {
/* 089 */           final int serializefromobject_numElements = serializefromobject_value.numElements();
/* 090 */           serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 4);
/* 091 */
/* 092 */           for (int serializefromobject_index = 0; serializefromobject_index < serializefromobject_numElements; serializefromobject_index++) {
/* 093 */             if (serializefromobject_value.isNullAt(serializefromobject_index)) {
/* 094 */               serializefromobject_arrayWriter.setNullInt(serializefromobject_index);
/* 095 */             } else {
/* 096 */               final int serializefromobject_element = serializefromobject_value.getInt(serializefromobject_index);
/* 097 */               serializefromobject_arrayWriter.write(serializefromobject_index, serializefromobject_element);
/* 098 */             }
/* 099 */           }
/* 100 */         }
/* 101 */
/* 102 */         serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
/* 103 */       }
/* 104 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
/* 105 */       append(serializefromobject_result);
/* 106 */       if (shouldStop()) return;
/* 107 */     }
/* 108 */   }
/* 109 */ }
```

Generated code after applying this PR

``` java
/* 035 */   protected void processNext() throws java.io.IOException {
/* 036 */     while (inputadapter_input.hasNext()) {
/* 037 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 038 */       int[] inputadapter_value = (int[])inputadapter_row.get(0, null);
/* 039 */
/* 040 */       Object mapelements_obj = ((Expression) references[0]).eval(null);
/* 041 */       scala.Function1 mapelements_value1 = (scala.Function1) mapelements_obj;
/* 042 */
/* 043 */       boolean mapelements_isNull = false || false;
/* 044 */       int[] mapelements_value = null;
/* 045 */       if (!mapelements_isNull) {
/* 046 */         Object mapelements_funcResult = null;
/* 047 */         mapelements_funcResult = mapelements_value1.apply(inputadapter_value);
/* 048 */         if (mapelements_funcResult == null) {
/* 049 */           mapelements_isNull = true;
/* 050 */         } else {
/* 051 */           mapelements_value = (int[]) mapelements_funcResult;
/* 052 */         }
/* 053 */
/* 054 */       }
/* 055 */       mapelements_isNull = mapelements_value == null;
/* 056 */
/* 057 */       boolean serializefromobject_isNull = mapelements_isNull;
/* 058 */       final ArrayData serializefromobject_value = serializefromobject_isNull ? null : org.apache.spark.sql.catalyst.expressions.UnsafeArrayData.fromPrimitiveArray(mapelements_value);
/* 059 */       serializefromobject_isNull = serializefromobject_value == null;
/* 060 */       serializefromobject_holder.reset();
/* 061 */
/* 062 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 063 */
/* 064 */       if (serializefromobject_isNull) {
/* 065 */         serializefromobject_rowWriter.setNullAt(0);
/* 066 */       } else {
/* 067 */         // Remember the current cursor so that we can calculate how many bytes are
/* 068 */         // written later.
/* 069 */         final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
/* 070 */
/* 071 */         if (serializefromobject_value instanceof UnsafeArrayData) {
/* 072 */           final int serializefromobject_sizeInBytes = ((UnsafeArrayData) serializefromobject_value).getSizeInBytes();
/* 073 */           // grow the global buffer before writing data.
/* 074 */           serializefromobject_holder.grow(serializefromobject_sizeInBytes);
/* 075 */           ((UnsafeArrayData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 076 */           serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
/* 077 */
/* 078 */         } else {
/* 079 */           final int serializefromobject_numElements = serializefromobject_value.numElements();
/* 080 */           serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 4);
/* 081 */
/* 082 */           for (int serializefromobject_index = 0; serializefromobject_index < serializefromobject_numElements; serializefromobject_index++) {
/* 083 */             if (serializefromobject_value.isNullAt(serializefromobject_index)) {
/* 084 */               serializefromobject_arrayWriter.setNullInt(serializefromobject_index);
/* 085 */             } else {
/* 086 */               final int serializefromobject_element = serializefromobject_value.getInt(serializefromobject_index);
/* 087 */               serializefromobject_arrayWriter.write(serializefromobject_index, serializefromobject_element);
/* 088 */             }
/* 089 */           }
/* 090 */         }
/* 091 */
/* 092 */         serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
/* 093 */       }
/* 094 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
/* 095 */       append(serializefromobject_result);
/* 096 */       if (shouldStop()) return;
/* 097 */     }
/* 098 */   }
/* 099 */ }
```
## How was this patch tested?

Added a test in `DatasetSuite`, `RowEncoderSuite`, and `CatalystTypeConvertersSuite`

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #15044 from kiszk/SPARK-17490.
2016-11-08 00:14:57 +01:00
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
Reynold Xin 9db06c442c [SPARK-18296][SQL] Use consistent naming for expression test suites
## What changes were proposed in this pull request?
We have an undocumented naming convention to call expression unit tests ExpressionsSuite, and the end-to-end tests FunctionsSuite. It'd be great to make all test suites consistent with this naming convention.

## How was this patch tested?
This is a test-only naming change.

Author: Reynold Xin <rxin@databricks.com>

Closes #15793 from rxin/SPARK-18296.
2016-11-06 22:44:55 -08:00
Wenchen Fan 46b2e49993 [SPARK-18173][SQL] data source tables should support truncating partition
## What changes were proposed in this pull request?

Previously `TRUNCATE TABLE ... PARTITION` will always truncate the whole table for data source tables, this PR fixes it and improve `InMemoryCatalog` to make this command work with it.
## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15688 from cloud-fan/truncate.
2016-11-06 18:57:13 -08:00
hyukjinkwon 340f09d100
[SPARK-17854][SQL] rand/randn allows null/long as input seed
## What changes were proposed in this pull request?

This PR proposes `rand`/`randn` accept `null` as input in Scala/SQL and `LongType` as input in SQL. In this case, it treats the values as `0`.

So, this PR includes both changes below:
- `null` support

  It seems MySQL also accepts this.

  ``` sql
  mysql> select rand(0);
  +---------------------+
  | rand(0)             |
  +---------------------+
  | 0.15522042769493574 |
  +---------------------+
  1 row in set (0.00 sec)

  mysql> select rand(NULL);
  +---------------------+
  | rand(NULL)          |
  +---------------------+
  | 0.15522042769493574 |
  +---------------------+
  1 row in set (0.00 sec)
  ```

  and also Hive does according to [HIVE-14694](https://issues.apache.org/jira/browse/HIVE-14694)

  So the codes below:

  ``` scala
  spark.range(1).selectExpr("rand(null)").show()
  ```

  prints..

  **Before**

  ```
    Input argument to rand must be an integer literal.;; line 1 pos 0
  org.apache.spark.sql.AnalysisException: Input argument to rand must be an integer literal.;; line 1 pos 0
  at org.apache.spark.sql.catalyst.analysis.FunctionRegistry$$anonfun$5.apply(FunctionRegistry.scala:465)
  at org.apache.spark.sql.catalyst.analysis.FunctionRegistry$$anonfun$5.apply(FunctionRegistry.scala:444)
  ```

  **After**

  ```
    +-----------------------+
    |rand(CAST(NULL AS INT))|
    +-----------------------+
    |    0.13385709732307427|
    +-----------------------+
  ```
- `LongType` support in SQL.

  In addition, it make the function allows to take `LongType` consistently within Scala/SQL.

  In more details, the codes below:

  ``` scala
  spark.range(1).select(rand(1), rand(1L)).show()
  spark.range(1).selectExpr("rand(1)", "rand(1L)").show()
  ```

  prints..

  **Before**

  ```
  +------------------+------------------+
  |           rand(1)|           rand(1)|
  +------------------+------------------+
  |0.2630967864682161|0.2630967864682161|
  +------------------+------------------+

  Input argument to rand must be an integer literal.;; line 1 pos 0
  org.apache.spark.sql.AnalysisException: Input argument to rand must be an integer literal.;; line 1 pos 0
  at org.apache.spark.sql.catalyst.analysis.FunctionRegistry$$anonfun$5.apply(FunctionRegistry.scala:465)
  at
  ```

  **After**

  ```
  +------------------+------------------+
  |           rand(1)|           rand(1)|
  +------------------+------------------+
  |0.2630967864682161|0.2630967864682161|
  +------------------+------------------+

  +------------------+------------------+
  |           rand(1)|           rand(1)|
  +------------------+------------------+
  |0.2630967864682161|0.2630967864682161|
  +------------------+------------------+
  ```
## How was this patch tested?

Unit tests in `DataFrameSuite.scala` and `RandomSuite.scala`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15432 from HyukjinKwon/SPARK-17854.
2016-11-06 14:11:37 +00:00
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
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
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
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
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
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
hyukjinkwon 7eb2ca8e33 [SPARK-17963][SQL][DOCUMENTATION] Add examples (extend) in each expression and improve documentation
## What changes were proposed in this pull request?

This PR proposes to change the documentation for functions. Please refer the discussion from https://github.com/apache/spark/pull/15513

The changes include
- Re-indent the documentation
- Add examples/arguments in `extended` where the arguments are multiple or specific format (e.g. xml/ json).

For examples, the documentation was updated as below:
### Functions with single line usage

**Before**
- `pow`

  ``` sql
  Usage: pow(x1, x2) - Raise x1 to the power of x2.
  Extended Usage:
  > SELECT pow(2, 3);
   8.0
  ```
- `current_timestamp`

  ``` sql
  Usage: current_timestamp() - Returns the current timestamp at the start of query evaluation.
  Extended Usage:
  No example for current_timestamp.
  ```

**After**
- `pow`

  ``` sql
  Usage: pow(expr1, expr2) - Raises `expr1` to the power of `expr2`.
  Extended Usage:
      Examples:
        > SELECT pow(2, 3);
         8.0
  ```

- `current_timestamp`

  ``` sql
  Usage: current_timestamp() - Returns the current timestamp at the start of query evaluation.
  Extended Usage:
      No example/argument for current_timestamp.
  ```
### Functions with (already) multiple line usage

**Before**
- `approx_count_distinct`

  ``` sql
  Usage: approx_count_distinct(expr) - Returns the estimated cardinality by HyperLogLog++.
      approx_count_distinct(expr, relativeSD=0.05) - Returns the estimated cardinality by HyperLogLog++
        with relativeSD, the maximum estimation error allowed.

  Extended Usage:
  No example for approx_count_distinct.
  ```
- `percentile_approx`

  ``` sql
  Usage:
        percentile_approx(col, percentage [, accuracy]) - Returns the approximate percentile value of numeric
        column `col` at the given percentage. The value of percentage must be between 0.0
        and 1.0. The `accuracy` parameter (default: 10000) is a positive integer literal which
        controls approximation accuracy at the cost of memory. Higher value of `accuracy` yields
        better accuracy, `1.0/accuracy` is the relative error of the approximation.

        percentile_approx(col, array(percentage1 [, percentage2]...) [, accuracy]) - Returns the approximate
        percentile array of column `col` at the given percentage array. Each value of the
        percentage array must be between 0.0 and 1.0. The `accuracy` parameter (default: 10000) is
        a positive integer literal which controls approximation accuracy at the cost of memory.
        Higher value of `accuracy` yields better accuracy, `1.0/accuracy` is the relative error of
        the approximation.

  Extended Usage:
  No example for percentile_approx.
  ```

**After**
- `approx_count_distinct`

  ``` sql
  Usage:
      approx_count_distinct(expr[, relativeSD]) - Returns the estimated cardinality by HyperLogLog++.
        `relativeSD` defines the maximum estimation error allowed.

  Extended Usage:
      No example/argument for approx_count_distinct.
  ```

- `percentile_approx`

  ``` sql
  Usage:
      percentile_approx(col, percentage [, accuracy]) - Returns the approximate percentile value of numeric
        column `col` at the given percentage. The value of percentage must be between 0.0
        and 1.0. The `accuracy` parameter (default: 10000) is a positive numeric literal which
        controls approximation accuracy at the cost of memory. Higher value of `accuracy` yields
        better accuracy, `1.0/accuracy` is the relative error of the approximation.
        When `percentage` is an array, each value of the percentage array must be between 0.0 and 1.0.
        In this case, returns the approximate percentile array of column `col` at the given
        percentage array.

  Extended Usage:
      Examples:
        > SELECT percentile_approx(10.0, array(0.5, 0.4, 0.1), 100);
         [10.0,10.0,10.0]
        > SELECT percentile_approx(10.0, 0.5, 100);
         10.0
  ```
## How was this patch tested?

Manually tested

**When examples are multiple**

``` sql
spark-sql> describe function extended reflect;
Function: reflect
Class: org.apache.spark.sql.catalyst.expressions.CallMethodViaReflection
Usage: reflect(class, method[, arg1[, arg2 ..]]) - Calls a method with reflection.
Extended Usage:
    Examples:
      > SELECT reflect('java.util.UUID', 'randomUUID');
       c33fb387-8500-4bfa-81d2-6e0e3e930df2
      > SELECT reflect('java.util.UUID', 'fromString', 'a5cf6c42-0c85-418f-af6c-3e4e5b1328f2');
       a5cf6c42-0c85-418f-af6c-3e4e5b1328f2
```

**When `Usage` is in single line**

``` sql
spark-sql> describe function extended min;
Function: min
Class: org.apache.spark.sql.catalyst.expressions.aggregate.Min
Usage: min(expr) - Returns the minimum value of `expr`.
Extended Usage:
    No example/argument for min.
```

**When `Usage` is already in multiple lines**

``` sql
spark-sql> describe function extended percentile_approx;
Function: percentile_approx
Class: org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile
Usage:
    percentile_approx(col, percentage [, accuracy]) - Returns the approximate percentile value of numeric
      column `col` at the given percentage. The value of percentage must be between 0.0
      and 1.0. The `accuracy` parameter (default: 10000) is a positive numeric literal which
      controls approximation accuracy at the cost of memory. Higher value of `accuracy` yields
      better accuracy, `1.0/accuracy` is the relative error of the approximation.
      When `percentage` is an array, each value of the percentage array must be between 0.0 and 1.0.
      In this case, returns the approximate percentile array of column `col` at the given
      percentage array.

Extended Usage:
    Examples:
      > SELECT percentile_approx(10.0, array(0.5, 0.4, 0.1), 100);
       [10.0,10.0,10.0]
      > SELECT percentile_approx(10.0, 0.5, 100);
       10.0
```

**When example/argument is missing**

``` sql
spark-sql> describe function extended rank;
Function: rank
Class: org.apache.spark.sql.catalyst.expressions.Rank
Usage:
    rank() - Computes the rank of a value in a group of values. The result is one plus the number
      of rows preceding or equal to the current row in the ordering of the partition. The values
      will produce gaps in the sequence.

Extended Usage:
    No example/argument for rank.
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15677 from HyukjinKwon/SPARK-17963-1.
2016-11-02 20:56:30 -07:00
Wenchen Fan 3a1bc6f478 [SPARK-17470][SQL] unify path for data source table and locationUri for hive serde table
## What changes were proposed in this pull request?

Due to a limitation of hive metastore(table location must be directory path, not file path), we always store `path` for data source table in storage properties, instead of the `locationUri` field. However, we should not expose this difference to `CatalogTable` level, but just treat it as a hack in `HiveExternalCatalog`, like we store table schema of data source table in table properties.

This PR unifies `path` and `locationUri` outside of `HiveExternalCatalog`, both data source table and hive serde table should use the `locationUri` field.

This PR also unifies the way we handle default table location for managed table. Previously, the default table location of hive serde managed table is set by external catalog, but the one of data source table is set by command. After this PR, we follow the hive way and the default table location is always set by external catalog.

For managed non-file-based tables, we will assign a default table location and create an empty directory for it, the table location will be removed when the table is dropped. This is reasonable as metastore doesn't care about whether a table is file-based or not, and an empty table directory has no harm.
For external non-file-based tables, ideally we can omit the table location, but due to a hive metastore issue, we will assign a random location to it, and remove it right after the table is created. See SPARK-15269 for more details. This is fine as it's well isolated in `HiveExternalCatalog`.

To keep the existing behaviour of the `path` option, in this PR we always add the `locationUri` to storage properties using key `path`, before passing storage properties to `DataSource` as data source options.
## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15024 from cloud-fan/path.
2016-11-02 18:05:14 -07:00
Reynold Xin fd90541c35 [SPARK-18214][SQL] Simplify RuntimeReplaceable type coercion
## What changes were proposed in this pull request?
RuntimeReplaceable is used to create aliases for expressions, but the way it deals with type coercion is pretty weird (each expression is responsible for how to handle type coercion, which does not obey the normal implicit type cast rules).

This patch simplifies its handling by allowing the analyzer to traverse into the actual expression of a RuntimeReplaceable.

## How was this patch tested?
- Correctness should be guaranteed by existing unit tests already
- Removed SQLCompatibilityFunctionSuite and moved it sql-compatibility-functions.sql
- Added a new test case in sql-compatibility-functions.sql for verifying explain behavior.

Author: Reynold Xin <rxin@databricks.com>

Closes #15723 from rxin/SPARK-18214.
2016-11-02 15:53:02 -07:00
Xiangrui Meng 02f203107b [SPARK-14393][SQL] values generated by non-deterministic functions shouldn't change after coalesce or union
## What changes were proposed in this pull request?

When a user appended a column using a "nondeterministic" function to a DataFrame, e.g., `rand`, `randn`, and `monotonically_increasing_id`, the expected semantic is the following:
- The value in each row should remain unchanged, as if we materialize the column immediately, regardless of later DataFrame operations.

However, since we use `TaskContext.getPartitionId` to get the partition index from the current thread, the values from nondeterministic columns might change if we call `union` or `coalesce` after. `TaskContext.getPartitionId` returns the partition index of the current Spark task, which might not be the corresponding partition index of the DataFrame where we defined the column.

See the unit tests below or JIRA for examples.

This PR uses the partition index from `RDD.mapPartitionWithIndex` instead of `TaskContext` and fixes the partition initialization logic in whole-stage codegen, normal codegen, and codegen fallback. `initializeStatesForPartition(partitionIndex: Int)` was added to `Projection`, `Nondeterministic`, and `Predicate` (codegen) and initialized right after object creation in `mapPartitionWithIndex`. `newPredicate` now returns a `Predicate` instance rather than a function for proper initialization.
## How was this patch tested?

Unit tests. (Actually I'm not very confident that this PR fixed all issues without introducing new ones ...)

cc: rxin davies

Author: Xiangrui Meng <meng@databricks.com>

Closes #15567 from mengxr/SPARK-14393.
2016-11-02 11:41:49 -07:00
Takeshi YAMAMURO 4af0ce2d96 [SPARK-17683][SQL] Support ArrayType in Literal.apply
## What changes were proposed in this pull request?

This pr is to add pattern-matching entries for array data in `Literal.apply`.
## How was this patch tested?

Added tests in `LiteralExpressionSuite`.

Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes #15257 from maropu/SPARK-17683.
2016-11-02 11:29:26 -07:00
eyal farago f151bd1af8 [SPARK-16839][SQL] Simplify Struct creation code path
## What changes were proposed in this pull request?

Simplify struct creation, especially the aspect of `CleanupAliases` which missed some aliases when handling trees created by `CreateStruct`.

This PR includes:

1. A failing test (create struct with nested aliases, some of the aliases survive `CleanupAliases`).
2. A fix that transforms `CreateStruct` into a `CreateNamedStruct` constructor, effectively eliminating `CreateStruct` from all expression trees.
3. A `NamePlaceHolder` used by `CreateStruct` when column names cannot be extracted from unresolved `NamedExpression`.
4. A new Analyzer rule that resolves `NamePlaceHolder` into a string literal once the `NamedExpression` is resolved.
5. `CleanupAliases` code was simplified as it no longer has to deal with `CreateStruct`'s top level columns.

## How was this patch tested?
Running all tests-suits in package org.apache.spark.sql, especially including the analysis suite, making sure added test initially fails, after applying suggested fix rerun the entire analysis package successfully.

Modified few tests that expected `CreateStruct` which is now transformed into `CreateNamedStruct`.

Author: eyal farago <eyal farago>
Author: Herman van Hovell <hvanhovell@databricks.com>
Author: eyal farago <eyal.farago@gmail.com>
Author: Eyal Farago <eyal.farago@actimize.com>
Author: Hyukjin Kwon <gurwls223@gmail.com>
Author: eyalfa <eyal.farago@gmail.com>

Closes #15718 from hvanhovell/SPARK-16839-2.
2016-11-02 11:12:20 +01:00
Sean Owen 9c8deef64e
[SPARK-18076][CORE][SQL] Fix default Locale used in DateFormat, NumberFormat to Locale.US
## What changes were proposed in this pull request?

Fix `Locale.US` for all usages of `DateFormat`, `NumberFormat`
## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #15610 from srowen/SPARK-18076.
2016-11-02 09:39:15 +00:00
Eric Liang abefe2ec42 [SPARK-18183][SPARK-18184] Fix INSERT [INTO|OVERWRITE] TABLE ... PARTITION for Datasource tables
## What changes were proposed in this pull request?

There are a couple issues with the current 2.1 behavior when inserting into Datasource tables with partitions managed by Hive.

(1) OVERWRITE TABLE ... PARTITION will actually overwrite the entire table instead of just the specified partition.
(2) INSERT|OVERWRITE does not work with partitions that have custom locations.

This PR fixes both of these issues for Datasource tables managed by Hive. The behavior for legacy tables or when `manageFilesourcePartitions = false` is unchanged.

There is one other issue in that INSERT OVERWRITE with dynamic partitions will overwrite the entire table instead of just the updated partitions, but this behavior is pretty complicated to implement for Datasource tables. We should address that in a future release.

## How was this patch tested?

Unit tests.

Author: Eric Liang <ekl@databricks.com>

Closes #15705 from ericl/sc-4942.
2016-11-02 14:15:10 +08:00
hyukjinkwon 01dd008301 [SPARK-17764][SQL] Add to_json supporting to convert nested struct column to JSON string
## What changes were proposed in this pull request?

This PR proposes to add `to_json` function in contrast with `from_json` in Scala, Java and Python.

It'd be useful if we can convert a same column from/to json. Also, some datasources do not support nested types. If we are forced to save a dataframe into those data sources, we might be able to work around by this function.

The usage is as below:

``` scala
val df = Seq(Tuple1(Tuple1(1))).toDF("a")
df.select(to_json($"a").as("json")).show()
```

``` bash
+--------+
|    json|
+--------+
|{"_1":1}|
+--------+
```
## How was this patch tested?

Unit tests in `JsonFunctionsSuite` and `JsonExpressionsSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15354 from HyukjinKwon/SPARK-17764.
2016-11-01 12:46:41 -07:00
jiangxingbo d0272b4365 [SPARK-18148][SQL] Misleading Error Message for Aggregation Without Window/GroupBy
## What changes were proposed in this pull request?

Aggregation Without Window/GroupBy expressions will fail in `checkAnalysis`, the error message is a bit misleading, we should generate a more specific error message for this case.

For example,

```
spark.read.load("/some-data")
  .withColumn("date_dt", to_date($"date"))
  .withColumn("year", year($"date_dt"))
  .withColumn("week", weekofyear($"date_dt"))
  .withColumn("user_count", count($"userId"))
  .withColumn("daily_max_in_week", max($"user_count").over(weeklyWindow))
)
```

creates the following output:

```
org.apache.spark.sql.AnalysisException: expression '`randomColumn`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.;
```

In the error message above, `randomColumn` doesn't appear in the query(acturally it's added by function `withColumn`), so the message is not enough for the user to address the problem.
## How was this patch tested?

Manually test

Before:

```
scala> spark.sql("select col, count(col) from tbl")
org.apache.spark.sql.AnalysisException: expression 'tbl.`col`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.;;
```

After:

```
scala> spark.sql("select col, count(col) from tbl")
org.apache.spark.sql.AnalysisException: grouping expressions sequence is empty, and 'tbl.`col`' is not an aggregate function. Wrap '(count(col#231L) AS count(col)#239L)' in windowing function(s) or wrap 'tbl.`col`' in first() (or first_value) if you don't care which value you get.;;
```

Also add new test sqls in `group-by.sql`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15672 from jiangxb1987/groupBy-empty.
2016-11-01 11:25:11 -07:00
Herman van Hovell 0cba535af3 Revert "[SPARK-16839][SQL] redundant aliases after cleanupAliases"
This reverts commit 5441a6269e.
2016-11-01 17:30:37 +01:00
eyal farago 5441a6269e [SPARK-16839][SQL] redundant aliases after cleanupAliases
## What changes were proposed in this pull request?

Simplify struct creation, especially the aspect of `CleanupAliases` which missed some aliases when handling trees created by `CreateStruct`.

This PR includes:

1. A failing test (create struct with nested aliases, some of the aliases survive `CleanupAliases`).
2. A fix that transforms `CreateStruct` into a `CreateNamedStruct` constructor, effectively eliminating `CreateStruct` from all expression trees.
3. A `NamePlaceHolder` used by `CreateStruct` when column names cannot be extracted from unresolved `NamedExpression`.
4. A new Analyzer rule that resolves `NamePlaceHolder` into a string literal once the `NamedExpression` is resolved.
5. `CleanupAliases` code was simplified as it no longer has to deal with `CreateStruct`'s top level columns.

## How was this patch tested?

running all tests-suits in package org.apache.spark.sql, especially including the analysis suite, making sure added test initially fails, after applying suggested fix rerun the entire analysis package successfully.

modified few tests that expected `CreateStruct` which is now transformed into `CreateNamedStruct`.

Credit goes to hvanhovell for assisting with this PR.

Author: eyal farago <eyal farago>
Author: eyal farago <eyal.farago@gmail.com>
Author: Herman van Hovell <hvanhovell@databricks.com>
Author: Eyal Farago <eyal.farago@actimize.com>
Author: Hyukjin Kwon <gurwls223@gmail.com>
Author: eyalfa <eyal.farago@gmail.com>

Closes #14444 from eyalfa/SPARK-16839_redundant_aliases_after_cleanupAliases.
2016-11-01 17:12:20 +01:00
Herman van Hovell f7c145d8ce [SPARK-17996][SQL] Fix unqualified catalog.getFunction(...)
## What changes were proposed in this pull request?

Currently an unqualified `getFunction(..)`call returns a wrong result; the returned function is shown as temporary function without a database. For example:

```
scala> sql("create function fn1 as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFAbs'")
res0: org.apache.spark.sql.DataFrame = []

scala> spark.catalog.getFunction("fn1")
res1: org.apache.spark.sql.catalog.Function = Function[name='fn1', className='org.apache.hadoop.hive.ql.udf.generic.GenericUDFAbs', isTemporary='true']
```

This PR fixes this by adding database information to ExpressionInfo (which is used to store the function information).
## How was this patch tested?

Added more thorough tests to `CatalogSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #15542 from hvanhovell/SPARK-17996.
2016-11-01 15:41:45 +01:00
wangzhenhua cb80edc263
[SPARK-18111][SQL] Wrong ApproximatePercentile answer when multiple records have the minimum value
## What changes were proposed in this pull request?

When multiple records have the minimum value, the answer of ApproximatePercentile is wrong.
## How was this patch tested?

add a test case

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #15641 from wzhfy/percentile.
2016-11-01 13:11:24 +00:00
Eric Liang ccb1154304 [SPARK-17970][SQL] store partition spec in metastore for data source table
## What changes were proposed in this pull request?

We should follow hive table and also store partition spec in metastore for data source table.
This brings 2 benefits:

1. It's more flexible to manage the table data files, as users can use `ADD PARTITION`, `DROP PARTITION` and `RENAME PARTITION`
2. We don't need to cache all file status for data source table anymore.

## How was this patch tested?

existing tests.

Author: Eric Liang <ekl@databricks.com>
Author: Michael Allman <michael@videoamp.com>
Author: Eric Liang <ekhliang@gmail.com>
Author: Wenchen Fan <wenchen@databricks.com>

Closes #15515 from cloud-fan/partition.
2016-10-27 14:22:30 -07:00
ALeksander Eskilson f1aeed8b02 [SPARK-17770][CATALYST] making ObjectType public
## What changes were proposed in this pull request?

In order to facilitate the writing of additional Encoders, I proposed opening up the ObjectType SQL DataType. This DataType is used extensively in the JavaBean Encoder, but would also be useful in writing other custom encoders.

As mentioned by marmbrus, it is understood that the Expressions API is subject to potential change.

## How was this patch tested?

The change only affects the visibility of the ObjectType class, and the existing SQL test suite still runs without error.

Author: ALeksander Eskilson <alek.eskilson@cerner.com>

Closes #15453 from bdrillard/master.
2016-10-26 18:03:31 -07:00
jiangxingbo fa7d9d7082 [SPARK-18063][SQL] Failed to infer constraints over multiple aliases
## What changes were proposed in this pull request?

The `UnaryNode.getAliasedConstraints` function fails to replace all expressions by their alias where constraints contains more than one expression to be replaced.
For example:
```
val tr = LocalRelation('a.int, 'b.string, 'c.int)
val multiAlias = tr.where('a === 'c + 10).select('a.as('x), 'c.as('y))
multiAlias.analyze.constraints
```
currently outputs:
```
ExpressionSet(Seq(
    IsNotNull(resolveColumn(multiAlias.analyze, "x")),
    IsNotNull(resolveColumn(multiAlias.analyze, "y"))
)
```
The constraint `resolveColumn(multiAlias.analyze, "x") === resolveColumn(multiAlias.analyze, "y") + 10)` is missing.

## How was this patch tested?

Add new test cases in `ConstraintPropagationSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15597 from jiangxb1987/alias-constraints.
2016-10-26 20:12:20 +02:00
jiangxingbo 3c023570b2 [SPARK-17733][SQL] InferFiltersFromConstraints rule never terminates for query
## What changes were proposed in this pull request?

The function `QueryPlan.inferAdditionalConstraints` and `UnaryNode.getAliasedConstraints` can produce a non-converging set of constraints for recursive functions. For instance, if we have two constraints of the form(where a is an alias):
`a = b, a = f(b, c)`
Applying both these rules in the next iteration would infer:
`f(b, c) = f(f(b, c), c)`
This process repeated, the iteration won't converge and the set of constraints will grow larger and larger until OOM.

~~To fix this problem, we collect alias from expressions and skip infer constraints if we are to transform an `Expression` to another which contains it.~~
To fix this problem, we apply additional check in `inferAdditionalConstraints`, when it's possible to generate recursive constraints, we skip generate that.

## How was this patch tested?

Add new testcase in `SQLQuerySuite`/`InferFiltersFromConstraintsSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15319 from jiangxb1987/constraints.
2016-10-26 17:09:48 +02:00
Wenchen Fan a21791e316 [SPARK-18070][SQL] binary operator should not consider nullability when comparing input types
## What changes were proposed in this pull request?

Binary operator requires its inputs to be of same type, but it should not consider nullability, e.g. `EqualTo` should be able to compare an element-nullable array and an element-non-nullable array.

## How was this patch tested?

a regression test in `DataFrameSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15606 from cloud-fan/type-bug.
2016-10-25 12:08:17 -07:00
Wenchen Fan 6f31833dbe [SPARK-18026][SQL] should not always lowercase partition columns of partition spec in parser
## What changes were proposed in this pull request?

Currently we always lowercase the partition columns of partition spec in parser, with the assumption that table partition columns are always lowercased.

However, this is not true for data source tables, which are case preserving. It's safe for now because data source tables don't store partition spec in metastore and don't support `ADD PARTITION`, `DROP PARTITION`, `RENAME PARTITION`, but we should make our code future-proof.

This PR makes partition spec case preserving at parser, and improve the `PreprocessTableInsertion` analyzer rule to normalize the partition columns in partition spec, w.r.t. the table partition columns.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15566 from cloud-fan/partition-spec.
2016-10-25 15:00:33 +08:00
Wenchen Fan 84a3399908 [SPARK-18028][SQL] simplify TableFileCatalog
## What changes were proposed in this pull request?

Simplify/cleanup TableFileCatalog:

1. pass a `CatalogTable` instead of `databaseName` and `tableName` into `TableFileCatalog`, so that we don't need to fetch table metadata from metastore again
2. In `TableFileCatalog.filterPartitions0`, DO NOT set `PartitioningAwareFileCatalog.BASE_PATH_PARAM`. According to the [classdoc](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala#L189-L209), the default value of `basePath` already satisfies our need. What's more, if we set this parameter, we may break the case 2 which is metioned in the classdoc.
3. add `equals` and `hashCode` to `TableFileCatalog`
4. add `SessionCatalog.listPartitionsByFilter` which handles case sensitivity.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15568 from cloud-fan/table-file-catalog.
2016-10-25 08:42:21 +08:00
CodingCat a81fba048f [SPARK-18058][SQL] Comparing column types ignoring Nullability in Union and SetOperation
## What changes were proposed in this pull request?

The PR tries to fix [SPARK-18058](https://issues.apache.org/jira/browse/SPARK-18058) which refers to a bug that the column types are compared with the extra care about Nullability in Union and SetOperation.

This PR converts the columns types by setting all fields as nullable before comparison

## How was this patch tested?

regular unit test cases

Author: CodingCat <zhunansjtu@gmail.com>

Closes #15595 from CodingCat/SPARK-18058.
2016-10-23 19:42:11 +02:00
Tejas Patil eff4aed1ac [SPARK-18035][SQL] Introduce performant and memory efficient APIs to create ArrayBasedMapData
## What changes were proposed in this pull request?

Jira: https://issues.apache.org/jira/browse/SPARK-18035

In HiveInspectors, I saw that converting Java map to Spark's `ArrayBasedMapData` spent quite sometime in buffer copying : https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala#L658

The reason being `map.toSeq` allocates a new buffer and copies the map entries to it: https://github.com/scala/scala/blob/2.11.x/src/library/scala/collection/MapLike.scala#L323

This copy is not needed as we get rid of it once we extract the key and value arrays.

Here is the call trace:

```
org.apache.spark.sql.hive.HiveInspectors$$anonfun$unwrapperFor$41.apply(HiveInspectors.scala:664)
scala.collection.AbstractMap.toSeq(Map.scala:59)
scala.collection.MapLike$class.toSeq(MapLike.scala:323)
scala.collection.AbstractMap.toBuffer(Map.scala:59)
scala.collection.MapLike$class.toBuffer(MapLike.scala:326)
scala.collection.AbstractTraversable.copyToBuffer(Traversable.scala:104)
scala.collection.TraversableOnce$class.copyToBuffer(TraversableOnce.scala:275)
scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:48)
scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:104)
scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59)
scala.collection.AbstractIterable.foreach(Iterable.scala:54)
scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
scala.collection.Iterator$class.foreach(Iterator.scala:893)
scala.collection.generic.Growable$$anonfun$$plus$plus$eq$1.apply(Growable.scala:59)
scala.collection.generic.Growable$$anonfun$$plus$plus$eq$1.apply(Growable.scala:59)
```

Also, earlier code was populating keys and values arrays separately by iterating twice. The PR avoids double iteration of the map and does it in one iteration.

EDIT: During code review, there were several more places in the code which were found to do similar thing. The PR dedupes those instances and introduces convenient APIs which are performant and memory efficient

## Performance gains

The number is subjective and depends on how many map columns are accessed in the query and average entries per map. For one the queries that I tried out, I saw 3% CPU savings (end-to-end) for the query.

## How was this patch tested?

This does not change the end result produced so relying on existing tests.

Author: Tejas Patil <tejasp@fb.com>

Closes #15573 from tejasapatil/SPARK-18035_avoid_toSeq.
2016-10-22 20:43:43 -07:00
Zheng RuiFeng a8ea4da8d0
[SPARK-17331][FOLLOWUP][ML][CORE] Avoid allocating 0-length arrays
## What changes were proposed in this pull request?

`Array[T]()` -> `Array.empty[T]` to avoid allocating 0-length arrays.
Use regex `find . -name '*.scala' | xargs -i bash -c 'egrep "Array\[[A-Za-z]+\]\(\)" -n {} && echo {}'` to find modification candidates.

cc srowen

## How was this patch tested?
existing tests

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #15564 from zhengruifeng/avoid_0_length_array.
2016-10-21 09:49:37 +01:00
Wenchen Fan 57e97fcbd6 [SPARK-18029][SQL] PruneFileSourcePartitions should not change the output of LogicalRelation
## What changes were proposed in this pull request?

In `PruneFileSourcePartitions`, we will replace the `LogicalRelation` with a pruned one. However, this replacement may change the output of the `LogicalRelation` if it doesn't have `expectedOutputAttributes`. This PR fixes it.

## How was this patch tested?

the new `PruneFileSourcePartitionsSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15569 from cloud-fan/partition-bug.
2016-10-21 12:27:53 +08:00
Koert Kuipers 84b245f2dd [SPARK-15780][SQL] Support mapValues on KeyValueGroupedDataset
## What changes were proposed in this pull request?

Add mapValues to KeyValueGroupedDataset

## How was this patch tested?

New test in DatasetSuite for groupBy function, mapValues, flatMap

Author: Koert Kuipers <koert@tresata.com>

Closes #13526 from koertkuipers/feat-keyvaluegroupeddataset-mapvalues.
2016-10-20 10:08:12 -07:00
Tejas Patil fb0894b3a8 [SPARK-17698][SQL] Join predicates should not contain filter clauses
## What changes were proposed in this pull request?

Jira : https://issues.apache.org/jira/browse/SPARK-17698

`ExtractEquiJoinKeys` is incorrectly using filter predicates as the join condition for joins. `canEvaluate` [0] tries to see if the an `Expression` can be evaluated using output of a given `Plan`. In case of filter predicates (eg. `a.id='1'`), the `Expression` passed for the right hand side (ie. '1' ) is a `Literal` which does not have any attribute references. Thus `expr.references` is an empty set which theoretically is a subset of any set. This leads to `canEvaluate` returning `true` and `a.id='1'` is treated as a join predicate. While this does not lead to incorrect results but in case of bucketed + sorted tables, we might miss out on avoiding un-necessary shuffle + sort. See example below:

[0] : https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala#L91

eg.

```
val df = (1 until 10).toDF("id").coalesce(1)
hc.sql("DROP TABLE IF EXISTS table1").collect
df.write.bucketBy(8, "id").sortBy("id").saveAsTable("table1")
hc.sql("DROP TABLE IF EXISTS table2").collect
df.write.bucketBy(8, "id").sortBy("id").saveAsTable("table2")

sqlContext.sql("""
  SELECT a.id, b.id
  FROM table1 a
  FULL OUTER JOIN table2 b
  ON a.id = b.id AND a.id='1' AND b.id='1'
""").explain(true)
```

BEFORE: This is doing shuffle + sort over table scan outputs which is not needed as both tables are bucketed and sorted on the same columns and have same number of buckets. This should be a single stage job.

```
SortMergeJoin [id#38, cast(id#38 as double), 1.0], [id#39, 1.0, cast(id#39 as double)], FullOuter
:- *Sort [id#38 ASC NULLS FIRST, cast(id#38 as double) ASC NULLS FIRST, 1.0 ASC NULLS FIRST], false, 0
:  +- Exchange hashpartitioning(id#38, cast(id#38 as double), 1.0, 200)
:     +- *FileScan parquet default.table1[id#38] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table1, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:int>
+- *Sort [id#39 ASC NULLS FIRST, 1.0 ASC NULLS FIRST, cast(id#39 as double) ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(id#39, 1.0, cast(id#39 as double), 200)
      +- *FileScan parquet default.table2[id#39] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table2, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:int>
```

AFTER :

```
SortMergeJoin [id#32], [id#33], FullOuter, ((cast(id#32 as double) = 1.0) && (cast(id#33 as double) = 1.0))
:- *FileScan parquet default.table1[id#32] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table1, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:int>
+- *FileScan parquet default.table2[id#33] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table2, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:int>
```

## How was this patch tested?

- Added a new test case for this scenario : `SPARK-17698 Join predicates should not contain filter clauses`
- Ran all the tests in `BucketedReadSuite`

Author: Tejas Patil <tejasp@fb.com>

Closes #15272 from tejasapatil/SPARK-17698_join_predicate_filter_clause.
2016-10-20 09:50:55 -07:00
hyukjinkwon 4b2011ec9d [SPARK-17989][SQL] Check ascendingOrder type in sort_array function rather than throwing ClassCastException
## What changes were proposed in this pull request?

This PR proposes to check the second argument, `ascendingOrder`  rather than throwing `ClassCastException` exception message.

```sql
select sort_array(array('b', 'd'), '1');
```

**Before**

```
16/10/19 13:16:08 ERROR SparkSQLDriver: Failed in [select sort_array(array('b', 'd'), '1')]
java.lang.ClassCastException: org.apache.spark.unsafe.types.UTF8String cannot be cast to java.lang.Boolean
	at scala.runtime.BoxesRunTime.unboxToBoolean(BoxesRunTime.java:85)
	at org.apache.spark.sql.catalyst.expressions.SortArray.nullSafeEval(collectionOperations.scala:185)
	at org.apache.spark.sql.catalyst.expressions.BinaryExpression.eval(Expression.scala:416)
	at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:50)
	at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:43)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:292)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:292)
	at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:74)
	at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:291)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:297)
```

**After**

```
Error in query: cannot resolve 'sort_array(array('b', 'd'), '1')' due to data type mismatch: Sort order in second argument requires a boolean literal.; line 1 pos 7;
```

## How was this patch tested?

Unit test in `DataFrameFunctionsSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15532 from HyukjinKwon/SPARK-17989.
2016-10-19 19:36:21 -07:00
Wenchen Fan 4329c5cea4 [SPARK-17873][SQL] ALTER TABLE RENAME TO should allow users to specify database in destination table name(but have to be same as source table)
## What changes were proposed in this pull request?

Unlike Hive, in Spark SQL, ALTER TABLE RENAME TO cannot move a table from one database to another(e.g. `ALTER TABLE db1.tbl RENAME TO db2.tbl2`), and will report error if the database in source table and destination table is different. So in #14955 , we forbid users to specify database of destination table in ALTER TABLE RENAME TO, to be consistent with other database systems and also make it easier to rename tables in non-current database, e.g. users can write `ALTER TABLE db1.tbl RENAME TO tbl2`, instead of `ALTER TABLE db1.tbl RENAME TO db1.tbl2`.

However, this is a breaking change. Users may already have queries that specify database of destination table in ALTER TABLE RENAME TO.

This PR reverts most of #14955 , and simplify the usage of ALTER TABLE RENAME TO by making database of source table the default database of destination table, instead of current database, so that users can still write `ALTER TABLE db1.tbl RENAME TO tbl2`, which is consistent with other databases like MySQL, Postgres, etc.

## How was this patch tested?

The added back tests and some new tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15434 from cloud-fan/revert.
2016-10-18 20:23:13 -07:00
gatorsmile d88a1bae6a [SPARK-17751][SQL] Remove spark.sql.eagerAnalysis and Output the Plan if Existed in AnalysisException
### What changes were proposed in this pull request?
Dataset always does eager analysis now. Thus, `spark.sql.eagerAnalysis` is not used any more. Thus, we need to remove it.

This PR also outputs the plan. Without the fix, the analysis error is like
```
cannot resolve '`k1`' given input columns: [k, v]; line 1 pos 12
```

After the fix, the analysis error becomes:
```
org.apache.spark.sql.AnalysisException: cannot resolve '`k1`' given input columns: [k, v]; line 1 pos 12;
'Project [unresolvedalias(CASE WHEN ('k1 = 2) THEN 22 WHEN ('k1 = 4) THEN 44 ELSE 0 END, None), v#6]
+- SubqueryAlias t
   +- Project [_1#2 AS k#5, _2#3 AS v#6]
      +- LocalRelation [_1#2, _2#3]
```

### How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15316 from gatorsmile/eagerAnalysis.
2016-10-17 11:33:06 -07:00
Weiqing Yang 56b0f5f4d1 [MINOR][SQL] Add prettyName for current_database function
## What changes were proposed in this pull request?
Added a `prettyname` for current_database function.

## How was this patch tested?
Manually.

Before:
```
scala> sql("select current_database()").show
+-----------------+
|currentdatabase()|
+-----------------+
|          default|
+-----------------+
```

After:
```
scala> sql("select current_database()").show
+------------------+
|current_database()|
+------------------+
|           default|
+------------------+
```

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #15506 from weiqingy/prettyName.
2016-10-16 22:38:30 -07:00
Michael Allman 6ce1b675ee [SPARK-16980][SQL] Load only catalog table partition metadata required to answer a query
(This PR addresses https://issues.apache.org/jira/browse/SPARK-16980.)

## What changes were proposed in this pull request?

In a new Spark session, when a partitioned Hive table is converted to use Spark's `HadoopFsRelation` in `HiveMetastoreCatalog`, metadata for every partition of that table are retrieved from the metastore and loaded into driver memory. In addition, every partition's metadata files are read from the filesystem to perform schema inference.

If a user queries such a table with predicates which prune that table's partitions, we would like to be able to answer that query without consulting partition metadata which are not involved in the query. When querying a table with a large number of partitions for some data from a small number of partitions (maybe even a single partition), the current conversion strategy is highly inefficient. I suspect this scenario is not uncommon in the wild.

In addition to being inefficient in running time, the current strategy is inefficient in its use of driver memory. When the sum of the number of partitions of all tables loaded in a driver reaches a certain level (somewhere in the tens of thousands), their cached data exhaust all driver heap memory in the default configuration. I suspect this scenario is less common (in that not too many deployments work with tables with tens of thousands of partitions), however this does illustrate how large the memory footprint of this metadata can be. With tables with hundreds or thousands of partitions, I would expect the `HiveMetastoreCatalog` table cache to represent a significant portion of the driver's heap space.

This PR proposes an alternative approach. Basically, it makes four changes:

1. It adds a new method, `listPartitionsByFilter` to the Catalyst `ExternalCatalog` trait which returns the partition metadata for a given sequence of partition pruning predicates.
1. It refactors the `FileCatalog` type hierarchy to include a new `TableFileCatalog` to efficiently return files only for partitions matching a sequence of partition pruning predicates.
1. It removes partition loading and caching from `HiveMetastoreCatalog`.
1. It adds a new Catalyst optimizer rule, `PruneFileSourcePartitions`, which applies a plan's partition-pruning predicates to prune out unnecessary partition files from a `HadoopFsRelation`'s underlying file catalog.

The net effect is that when a query over a partitioned Hive table is planned, the analyzer retrieves the table metadata from `HiveMetastoreCatalog`. As part of this operation, the `HiveMetastoreCatalog` builds a `HadoopFsRelation` with a `TableFileCatalog`. It does not load any partition metadata or scan any files. The optimizer prunes-away unnecessary table partitions by sending the partition-pruning predicates to the relation's `TableFileCatalog `. The `TableFileCatalog` in turn calls the `listPartitionsByFilter` method on its external catalog. This queries the Hive metastore, passing along those filters.

As a bonus, performing partition pruning during optimization leads to a more accurate relation size estimate. This, along with c481bdf, can lead to automatic, safe application of the broadcast optimization in a join where it might previously have been omitted.

## Open Issues

1. This PR omits partition metadata caching. I can add this once the overall strategy for the cold path is established, perhaps in a future PR.
1. This PR removes and omits partitioned Hive table schema reconciliation. As a result, it fails to find Parquet schema columns with upper case letters because of the Hive metastore's case-insensitivity. This issue may be fixed by #14750, but that PR appears to have stalled. ericl has contributed to this PR a workaround for Parquet wherein schema reconciliation occurs at query execution time instead of planning. Whether ORC requires a similar patch is an open issue.
1. This PR omits an implementation of `listPartitionsByFilter` for the `InMemoryCatalog`.
1. This PR breaks parquet log output redirection during query execution. I can work around this by running `Class.forName("org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$")` first thing in a Spark shell session, but I haven't figured out how to fix this properly.

## How was this patch tested?

The current Spark unit tests were run, and some ad-hoc tests were performed to validate that only the necessary partition metadata is loaded.

Author: Michael Allman <michael@videoamp.com>
Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekhliang@gmail.com>

Closes #14690 from mallman/spark-16980-lazy_partition_fetching.
2016-10-14 18:26:18 -07:00
Jeff Zhang f00df40cfe [SPARK-11775][PYSPARK][SQL] Allow PySpark to register Java UDF
Currently pyspark can only call the builtin java UDF, but can not call custom java UDF. It would be better to allow that. 2 benefits:
* Leverage the power of rich third party java library
* Improve the performance. Because if we use python UDF, python daemons will be started on worker which will affect the performance.

Author: Jeff Zhang <zjffdu@apache.org>

Closes #9766 from zjffdu/SPARK-11775.
2016-10-14 15:50:35 -07:00
Davies Liu da9aeb0fde [SPARK-17863][SQL] should not add column into Distinct
## What changes were proposed in this pull request?

We are trying to resolve the attribute in sort by pulling up some column for grandchild into child, but that's wrong when the child is Distinct, because the added column will change the behavior of Distinct, we should not do that.

## How was this patch tested?

Added regression test.

Author: Davies Liu <davies@databricks.com>

Closes #15489 from davies/order_distinct.
2016-10-14 14:45:20 -07:00
Wenchen Fan 2fb12b0a33 [SPARK-17903][SQL] MetastoreRelation should talk to external catalog instead of hive client
## What changes were proposed in this pull request?

`HiveExternalCatalog` should be the only interface to talk to the hive metastore. In `MetastoreRelation` we can just use `ExternalCatalog` instead of `HiveClient` to interact with hive metastore,  and add missing API in `ExternalCatalog`.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15460 from cloud-fan/relation.
2016-10-14 15:53:50 +08:00
Jakob Odersky 9dc0ca060d [SPARK-17368][SQL] Add support for value class serialization and deserialization
## What changes were proposed in this pull request?
Value classes were unsupported because catalyst data types were
obtained through reflection on erased types, which would resolve to a
value class' wrapped type and hence lead to unavailable methods during
code generation.

E.g. the following class
```scala
case class Foo(x: Int) extends AnyVal
```
would be seen as an `int` in catalyst and will cause instance cast failures when generated java code tries to treat it as a `Foo`.

This patch simply removes the erasure step when getting data types for
catalyst.

## How was this patch tested?
Additional tests in `ExpressionEncoderSuite`.

Author: Jakob Odersky <jakob@odersky.com>

Closes #15284 from jodersky/value-classes.
2016-10-13 17:48:09 -07:00
Tathagata Das 7106866c22 [SPARK-17731][SQL][STREAMING] Metrics for structured streaming
## What changes were proposed in this pull request?

Metrics are needed for monitoring structured streaming apps. Here is the design doc for implementing the necessary metrics.
https://docs.google.com/document/d/1NIdcGuR1B3WIe8t7VxLrt58TJB4DtipWEbj5I_mzJys/edit?usp=sharing

Specifically, this PR adds the following public APIs changes.

### New APIs
- `StreamingQuery.status` returns a `StreamingQueryStatus` object (renamed from `StreamingQueryInfo`, see later)

- `StreamingQueryStatus` has the following important fields
  - inputRate - Current rate (rows/sec) at which data is being generated by all the sources
  - processingRate - Current rate (rows/sec) at which the query is processing data from
                                  all the sources
  - ~~outputRate~~ - *Does not work with wholestage codegen*
  - latency - Current average latency between the data being available in source and the sink writing the corresponding output
  - sourceStatuses: Array[SourceStatus] - Current statuses of the sources
  - sinkStatus: SinkStatus - Current status of the sink
  - triggerStatus - Low-level detailed status of the last completed/currently active trigger
    - latencies - getOffset, getBatch, full trigger, wal writes
    - timestamps - trigger start, finish, after getOffset, after getBatch
    - numRows - input, output, state total/updated rows for aggregations

- `SourceStatus` has the following important fields
  - inputRate - Current rate (rows/sec) at which data is being generated by the source
  - processingRate - Current rate (rows/sec) at which the query is processing data from the source
  - triggerStatus - Low-level detailed status of the last completed/currently active trigger

- Python API for `StreamingQuery.status()`

### Breaking changes to existing APIs
**Existing direct public facing APIs**
- Deprecated direct public-facing APIs `StreamingQuery.sourceStatuses` and `StreamingQuery.sinkStatus` in favour of `StreamingQuery.status.sourceStatuses/sinkStatus`.
  - Branch 2.0 should have it deprecated, master should have it removed.

**Existing advanced listener APIs**
- `StreamingQueryInfo` renamed to `StreamingQueryStatus` for consistency with `SourceStatus`, `SinkStatus`
   - Earlier StreamingQueryInfo was used only in the advanced listener API, but now it is used in direct public-facing API (StreamingQuery.status)

- Field `queryInfo` in listener events `QueryStarted`, `QueryProgress`, `QueryTerminated` changed have name `queryStatus` and return type `StreamingQueryStatus`.

- Field `offsetDesc` in `SourceStatus` was Option[String], converted it to `String`.

- For `SourceStatus` and `SinkStatus` made constructor private instead of private[sql] to make them more java-safe. Instead added `private[sql] object SourceStatus/SinkStatus.apply()` which are harder to accidentally use in Java.

## How was this patch tested?

Old and new unit tests.
- Rate calculation and other internal logic of StreamMetrics tested by StreamMetricsSuite.
- New info in statuses returned through StreamingQueryListener is tested in StreamingQueryListenerSuite.
- New and old info returned through StreamingQuery.status is tested in StreamingQuerySuite.
- Source-specific tests for making sure input rows are counted are is source-specific test suites.
- Additional tests to test minor additions in LocalTableScanExec, StateStore, etc.

Metrics also manually tested using Ganglia sink

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #15307 from tdas/SPARK-17731.
2016-10-13 13:36:26 -07:00
Pete Robbins 84f149e414 [SPARK-17827][SQL] maxColLength type should be Int for String and Binary
## What changes were proposed in this pull request?
correct the expected type from Length function to be Int

## How was this patch tested?
Test runs on little endian and big endian platforms

Author: Pete Robbins <robbinspg@gmail.com>

Closes #15464 from robbinspg/SPARK-17827.
2016-10-13 11:26:30 -07:00
buzhihuojie 7222a25a11 minor doc fix for Row.scala
## What changes were proposed in this pull request?

minor doc fix for "getAnyValAs" in class Row

## How was this patch tested?

None.

(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Author: buzhihuojie <ren.weiluo@gmail.com>

Closes #15452 from david-weiluo-ren/minorDocFixForRow.
2016-10-12 22:51:54 -07:00
prigarg d5580ebaa0 [SPARK-17884][SQL] To resolve Null pointer exception when casting from empty string to interval type.
## What changes were proposed in this pull request?
This change adds a check in castToInterval method of Cast expression , such that if converted value is null , then isNull variable should be set to true.

Earlier, the expression Cast(Literal(), CalendarIntervalType) was throwing NullPointerException because of the above mentioned reason.

## How was this patch tested?
Added test case in CastSuite.scala

jira entry for detail: https://issues.apache.org/jira/browse/SPARK-17884

Author: prigarg <prigarg@adobe.com>

Closes #15449 from priyankagargnitk/SPARK-17884.
2016-10-12 10:14:45 -07:00
Wenchen Fan b9a147181d [SPARK-17720][SQL] introduce static SQL conf
## What changes were proposed in this pull request?

SQLConf is session-scoped and mutable. However, we do have the requirement for a static SQL conf, which is global and immutable, e.g. the `schemaStringThreshold` in `HiveExternalCatalog`, the flag to enable/disable hive support, the global temp view database in https://github.com/apache/spark/pull/14897.

Actually we've already implemented static SQL conf implicitly via `SparkConf`, this PR just make it explicit and expose it to users, so that they can see the config value via SQL command or `SparkSession.conf`, and forbid users to set/unset static SQL conf.

## How was this patch tested?

new tests in SQLConfSuite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15295 from cloud-fan/global-conf.
2016-10-11 20:27:08 -07:00
Liang-Chi Hsieh c8c090640a [SPARK-17821][SQL] Support And and Or in Expression Canonicalize
## What changes were proposed in this pull request?

Currently `Canonicalize` object doesn't support `And` and `Or`. So we can compare canonicalized form of predicates consistently. We should add the support.

## How was this patch tested?

Jenkins tests.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #15388 from viirya/canonicalize-and-or.
2016-10-11 16:06:40 +08:00
Reynold Xin 3694ba48f0 [SPARK-17864][SQL] Mark data type APIs as stable (not DeveloperApi)
## What changes were proposed in this pull request?
The data type API has not been changed since Spark 1.3.0, and is ready for graduation. This patch marks them as stable APIs using the new InterfaceStability annotation.

This patch also looks at the various files in the catalyst module (not the "package") and marks the remaining few classes appropriately as well.

## How was this patch tested?
This is an annotation change. No functional changes.

Author: Reynold Xin <rxin@databricks.com>

Closes #15426 from rxin/SPARK-17864.
2016-10-11 15:35:52 +08:00
Wenchen Fan 7388ad94d7 [SPARK-17338][SQL][FOLLOW-UP] add global temp view
## What changes were proposed in this pull request?

address post hoc review comments for https://github.com/apache/spark/pull/14897

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15424 from cloud-fan/global-temp-view.
2016-10-11 15:21:28 +08:00
Wenchen Fan 23ddff4b2b [SPARK-17338][SQL] add global temp view
## What changes were proposed in this pull request?

Global temporary view is a cross-session temporary view, which means it's shared among all sessions. Its lifetime is the lifetime of the Spark application, i.e. it will be automatically dropped when the application terminates. It's tied to a system preserved database `global_temp`(configurable via SparkConf), and we must use the qualified name to refer a global temp view, e.g. SELECT * FROM global_temp.view1.

changes for `SessionCatalog`:

1. add a new field `gloabalTempViews: GlobalTempViewManager`, to access the shared global temp views, and the global temp db name.
2. `createDatabase` will fail if users wanna create `global_temp`, which is system preserved.
3. `setCurrentDatabase` will fail if users wanna set `global_temp`, which is system preserved.
4. add `createGlobalTempView`, which is used in `CreateViewCommand` to create global temp views.
5. add `dropGlobalTempView`, which is used in `CatalogImpl` to drop global temp view.
6. add `alterTempViewDefinition`, which is used in `AlterViewAsCommand` to update the view definition for local/global temp views.
7. `renameTable`/`dropTable`/`isTemporaryTable`/`lookupRelation`/`getTempViewOrPermanentTableMetadata`/`refreshTable` will handle global temp views.

changes for SQL commands:

1. `CreateViewCommand`/`AlterViewAsCommand` is updated to support global temp views
2. `ShowTablesCommand` outputs a new column `database`, which is used to distinguish global and local temp views.
3. other commands can also handle global temp views if they call `SessionCatalog` APIs which accepts global temp views, e.g. `DropTableCommand`, `AlterTableRenameCommand`, `ShowColumnsCommand`, etc.

changes for other public API

1. add a new method `dropGlobalTempView` in `Catalog`
2. `Catalog.findTable` can find global temp view
3. add a new method `createGlobalTempView` in `Dataset`

## How was this patch tested?

new tests in `SQLViewSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14897 from cloud-fan/global-temp-view.
2016-10-10 15:48:57 +08:00
jiangxingbo 16590030c1 [SPARK-17741][SQL] Grammar to parse top level and nested data fields separately
## What changes were proposed in this pull request?

Currently we use the same rule to parse top level and nested data fields. For example:
```
create table tbl_x(
  id bigint,
  nested struct<col1:string,col2:string>
)
```
Shows both syntaxes. In this PR we split this rule in a top-level and nested rule.

Before this PR,
```
sql("CREATE TABLE my_tab(column1: INT)")
```
works fine.
After this PR, it will throw a `ParseException`:
```
scala> sql("CREATE TABLE my_tab(column1: INT)")
org.apache.spark.sql.catalyst.parser.ParseException:
no viable alternative at input 'CREATE TABLE my_tab(column1:'(line 1, pos 27)
```

## How was this patch tested?
Add new testcases in `SparkSqlParserSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15346 from jiangxb1987/cdt.
2016-10-09 22:00:54 -07:00
jiangxingbo 26fbca4806 [SPARK-17832][SQL] TableIdentifier.quotedString creates un-parseable names when name contains a backtick
## What changes were proposed in this pull request?

The `quotedString` method in `TableIdentifier` and `FunctionIdentifier` produce an illegal (un-parseable) name when the name contains a backtick. For example:
```
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser._
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
val complexName = TableIdentifier("`weird`table`name", Some("`d`b`1"))
parseTableIdentifier(complexName.unquotedString) // Does not work
parseTableIdentifier(complexName.quotedString) // Does not work
parseExpression(complexName.unquotedString) // Does not work
parseExpression(complexName.quotedString) // Does not work
```
We should handle the backtick properly to make `quotedString` parseable.

## How was this patch tested?
Add new testcases in `TableIdentifierParserSuite` and `ExpressionParserSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15403 from jiangxb1987/backtick.
2016-10-09 21:52:46 -07:00
Herman van Hovell 97594c29b7 [SPARK-17761][SQL] Remove MutableRow
## What changes were proposed in this pull request?
In practice we cannot guarantee that an `InternalRow` is immutable. This makes the `MutableRow` almost redundant. This PR folds `MutableRow` into `InternalRow`.

The code below illustrates the immutability issue with InternalRow:
```scala
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
val struct = new GenericMutableRow(1)
val row = InternalRow(struct, 1)
println(row)
scala> [[null], 1]
struct.setInt(0, 42)
println(row)
scala> [[42], 1]
```

This might be somewhat controversial, so feedback is appreciated.

## How was this patch tested?
Existing tests.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #15333 from hvanhovell/SPARK-17761.
2016-10-07 14:03:45 -07:00
Dongjoon Hyun 92b7e57280 [SPARK-17750][SQL] Fix CREATE VIEW with INTERVAL arithmetic.
## What changes were proposed in this pull request?

Currently, Spark raises `RuntimeException` when creating a view with timestamp with INTERVAL arithmetic like the following. The root cause is the arithmetic expression, `TimeAdd`, was transformed into `timeadd` function as a VIEW definition. This PR fixes the SQL definition of `TimeAdd` and `TimeSub` expressions.

```scala
scala> sql("CREATE TABLE dates (ts TIMESTAMP)")

scala> sql("CREATE VIEW view1 AS SELECT ts + INTERVAL 1 DAY FROM dates")
java.lang.RuntimeException: Failed to analyze the canonicalized SQL: ...
```

## How was this patch tested?

Pass Jenkins with a new testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15318 from dongjoon-hyun/SPARK-17750.
2016-10-06 09:42:30 -07:00
Herman van Hovell 5fd54b994e [SPARK-17758][SQL] Last returns wrong result in case of empty partition
## What changes were proposed in this pull request?
The result of the `Last` function can be wrong when the last partition processed is empty. It can return `null` instead of the expected value. For example, this can happen when we process partitions in the following order:
```
- Partition 1 [Row1, Row2]
- Partition 2 [Row3]
- Partition 3 []
```
In this case the `Last` function will currently return a null, instead of the value of `Row3`.

This PR fixes this by adding a `valueSet` flag to the `Last` function.

## How was this patch tested?
We only used end to end tests for `DeclarativeAggregateFunction`s. I have added an evaluator for these functions so we can tests them in catalyst. I have added a `LastTestSuite` to test the `Last` aggregate function.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #15348 from hvanhovell/SPARK-17758.
2016-10-05 16:05:30 -07:00
Dongjoon Hyun 6a05eb24d0 [SPARK-17328][SQL] Fix NPE with EXPLAIN DESCRIBE TABLE
## What changes were proposed in this pull request?

This PR fixes the following NPE scenario in two ways.

**Reported Error Scenario**
```scala
scala> sql("EXPLAIN DESCRIBE TABLE x").show(truncate = false)
INFO SparkSqlParser: Parsing command: EXPLAIN DESCRIBE TABLE x
java.lang.NullPointerException
```

- **DESCRIBE**: Extend `DESCRIBE` syntax to accept `TABLE`.
- **EXPLAIN**: Prevent NPE in case of the parsing failure of target statement, e.g., `EXPLAIN DESCRIBE TABLES x`.

## How was this patch tested?

Pass the Jenkins test with a new test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15357 from dongjoon-hyun/SPARK-17328.
2016-10-05 10:52:43 -07:00
Herman van Hovell 89516c1c4a [SPARK-17258][SQL] Parse scientific decimal literals as decimals
## What changes were proposed in this pull request?
Currently Spark SQL parses regular decimal literals (e.g. `10.00`) as decimals and scientific decimal literals (e.g. `10.0e10`) as doubles. The difference between the two confuses most users. This PR unifies the parsing behavior and also parses scientific decimal literals as decimals.

This implications in tests are limited to a single Hive compatibility test.

## How was this patch tested?
Updated tests in `ExpressionParserSuite` and `SQLQueryTestSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #14828 from hvanhovell/SPARK-17258.
2016-10-04 23:48:26 -07:00
Tejas Patil a99743d053 [SPARK-17495][SQL] Add Hash capability semantically equivalent to Hive's
## What changes were proposed in this pull request?

Jira : https://issues.apache.org/jira/browse/SPARK-17495

Spark internally uses Murmur3Hash for partitioning. This is different from the one used by Hive. For queries which use bucketing this leads to different results if one tries the same query on both engines. For us, we want users to have backward compatibility to that one can switch parts of applications across the engines without observing regressions.

This PR includes `HiveHash`, `HiveHashFunction`, `HiveHasher` which mimics Hive's hashing at https://github.com/apache/hive/blob/master/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java#L638

I am intentionally not introducing any usages of this hash function in rest of the code to keep this PR small. My eventual goal is to have Hive bucketing support in Spark. Once this PR gets in, I will make hash function pluggable in relevant areas (eg. `HashPartitioning`'s `partitionIdExpression` has Murmur3 hardcoded : https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala#L265)

## How was this patch tested?

Added `HiveHashSuite`

Author: Tejas Patil <tejasp@fb.com>

Closes #15047 from tejasapatil/SPARK-17495_hive_hash.
2016-10-04 18:59:31 -07:00
Takuya UESHIN b1b47274bf [SPARK-17702][SQL] Code generation including too many mutable states exceeds JVM size limit.
## What changes were proposed in this pull request?

Code generation including too many mutable states exceeds JVM size limit to extract values from `references` into fields in the constructor.
We should split the generated extractions in the constructor into smaller functions.

## How was this patch tested?

I added some tests to check if the generated codes for the expressions exceed or not.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #15275 from ueshin/issues/SPARK-17702.
2016-10-03 21:48:58 -07:00
Herman van Hovell 2bbecdec20 [SPARK-17753][SQL] Allow a complex expression as the input a value based case statement
## What changes were proposed in this pull request?
We currently only allow relatively simple expressions as the input for a value based case statement. Expressions like `case (a > 1) or (b = 2) when true then 1 when false then 0 end` currently fail. This PR adds support for such expressions.

## How was this patch tested?
Added a test to the ExpressionParserSuite.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #15322 from hvanhovell/SPARK-17753.
2016-10-03 19:32:59 -07:00
Zhenhua Wang 7bf9212764 [SPARK-17073][SQL] generate column-level statistics
## What changes were proposed in this pull request?

Generate basic column statistics for all the atomic types:
- numeric types: max, min, num of nulls, ndv (number of distinct values)
- date/timestamp types: they are also represented as numbers internally, so they have the same stats as above.
- string: avg length, max length, num of nulls, ndv
- binary: avg length, max length, num of nulls
- boolean: num of nulls, num of trues, num of falsies

Also support storing and loading these statistics.

One thing to notice:
We support analyzing columns independently, e.g.:
sql1: `ANALYZE TABLE src COMPUTE STATISTICS FOR COLUMNS key;`
sql2: `ANALYZE TABLE src COMPUTE STATISTICS FOR COLUMNS value;`
when running sql2 to collect column stats for `value`, we don’t remove stats of columns `key` which are analyzed in sql1 and not in sql2. As a result, **users need to guarantee consistency** between sql1 and sql2. If the table has been changed before sql2, users should re-analyze column `key` when they want to analyze column `value`:
`ANALYZE TABLE src COMPUTE STATISTICS FOR COLUMNS key, value;`

## How was this patch tested?

add unit tests

Author: Zhenhua Wang <wzh_zju@163.com>

Closes #15090 from wzhfy/colStats.
2016-10-03 10:12:02 -07:00
Dongjoon Hyun aef506e39a [SPARK-17739][SQL] Collapse adjacent similar Window operators
## What changes were proposed in this pull request?

Currently, Spark does not collapse adjacent windows with the same partitioning and sorting. This PR implements `CollapseWindow` optimizer to do the followings.

1. If the partition specs and order specs are the same, collapse into the parent.
2. If the partition specs are the same and one order spec is a prefix of the other, collapse to the more specific one.

For example:
```scala
val df = spark.range(1000).select($"id" % 100 as "grp", $"id", rand() as "col1", rand() as "col2")

// Add summary statistics for all columns
import org.apache.spark.sql.expressions.Window
val cols = Seq("id", "col1", "col2")
val window = Window.partitionBy($"grp").orderBy($"id")
val result = cols.foldLeft(df) { (base, name) =>
  base.withColumn(s"${name}_avg", avg(col(name)).over(window))
      .withColumn(s"${name}_stddev", stddev(col(name)).over(window))
      .withColumn(s"${name}_min", min(col(name)).over(window))
      .withColumn(s"${name}_max", max(col(name)).over(window))
}
```

**Before**
```scala
scala> result.explain
== Physical Plan ==
Window [max(col2#19) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_max#234], [grp#17L], [id#14L ASC NULLS FIRST]
+- Window [min(col2#19) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_min#216], [grp#17L], [id#14L ASC NULLS FIRST]
   +- Window [stddev_samp(col2#19) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_stddev#191], [grp#17L], [id#14L ASC NULLS FIRST]
      +- Window [avg(col2#19) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_avg#167], [grp#17L], [id#14L ASC NULLS FIRST]
         +- Window [max(col1#18) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_max#152], [grp#17L], [id#14L ASC NULLS FIRST]
            +- Window [min(col1#18) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_min#138], [grp#17L], [id#14L ASC NULLS FIRST]
               +- Window [stddev_samp(col1#18) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_stddev#117], [grp#17L], [id#14L ASC NULLS FIRST]
                  +- Window [avg(col1#18) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_avg#97], [grp#17L], [id#14L ASC NULLS FIRST]
                     +- Window [max(id#14L) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_max#86L], [grp#17L], [id#14L ASC NULLS FIRST]
                        +- Window [min(id#14L) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_min#76L], [grp#17L], [id#14L ASC NULLS FIRST]
                           +- *Project [grp#17L, id#14L, col1#18, col2#19, id_avg#26, id_stddev#42]
                              +- Window [stddev_samp(_w0#59) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_stddev#42], [grp#17L], [id#14L ASC NULLS FIRST]
                                 +- *Project [grp#17L, id#14L, col1#18, col2#19, id_avg#26, cast(id#14L as double) AS _w0#59]
                                    +- Window [avg(id#14L) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_avg#26], [grp#17L], [id#14L ASC NULLS FIRST]
                                       +- *Sort [grp#17L ASC NULLS FIRST, id#14L ASC NULLS FIRST], false, 0
                                          +- Exchange hashpartitioning(grp#17L, 200)
                                             +- *Project [(id#14L % 100) AS grp#17L, id#14L, rand(-6329949029880411066) AS col1#18, rand(-7251358484380073081) AS col2#19]
                                                +- *Range (0, 1000, step=1, splits=Some(8))
```

**After**
```scala
scala> result.explain
== Physical Plan ==
Window [max(col2#5) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_max#220, min(col2#5) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_min#202, stddev_samp(col2#5) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_stddev#177, avg(col2#5) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_avg#153, max(col1#4) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_max#138, min(col1#4) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_min#124, stddev_samp(col1#4) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_stddev#103, avg(col1#4) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_avg#83, max(id#0L) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_max#72L, min(id#0L) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_min#62L], [grp#3L], [id#0L ASC NULLS FIRST]
+- *Project [grp#3L, id#0L, col1#4, col2#5, id_avg#12, id_stddev#28]
   +- Window [stddev_samp(_w0#45) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_stddev#28], [grp#3L], [id#0L ASC NULLS FIRST]
      +- *Project [grp#3L, id#0L, col1#4, col2#5, id_avg#12, cast(id#0L as double) AS _w0#45]
         +- Window [avg(id#0L) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_avg#12], [grp#3L], [id#0L ASC NULLS FIRST]
            +- *Sort [grp#3L ASC NULLS FIRST, id#0L ASC NULLS FIRST], false, 0
               +- Exchange hashpartitioning(grp#3L, 200)
                  +- *Project [(id#0L % 100) AS grp#3L, id#0L, rand(6537478539664068821) AS col1#4, rand(-8961093871295252795) AS col2#5]
                     +- *Range (0, 1000, step=1, splits=Some(8))
```

## How was this patch tested?

Pass the Jenkins tests with a newly added testsuite.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15317 from dongjoon-hyun/SPARK-17739.
2016-09-30 21:05:06 -07:00
Takuya UESHIN 81455a9cd9 [SPARK-17703][SQL] Add unnamed version of addReferenceObj for minor objects.
## What changes were proposed in this pull request?

There are many minor objects in references, which are extracted to the generated class field, e.g. `errMsg` in `GetExternalRowField` or `ValidateExternalType`, but number of fields in class is limited so we should reduce the number.
This pr adds unnamed version of `addReferenceObj` for these minor objects not to store the object into field but refer it from the `references` field at the time of use.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #15276 from ueshin/issues/SPARK-17703.
2016-09-30 17:31:59 -07:00
Dongjoon Hyun 4ecc648ad7 [SPARK-17612][SQL] Support DESCRIBE table PARTITION SQL syntax
## What changes were proposed in this pull request?

This PR implements `DESCRIBE table PARTITION` SQL Syntax again. It was supported until Spark 1.6.2, but was dropped since 2.0.0.

**Spark 1.6.2**
```scala
scala> sql("CREATE TABLE partitioned_table (a STRING, b INT) PARTITIONED BY (c STRING, d STRING)")
res1: org.apache.spark.sql.DataFrame = [result: string]

scala> sql("ALTER TABLE partitioned_table ADD PARTITION (c='Us', d=1)")
res2: org.apache.spark.sql.DataFrame = [result: string]

scala> sql("DESC partitioned_table PARTITION (c='Us', d=1)").show(false)
+----------------------------------------------------------------+
|result                                                          |
+----------------------------------------------------------------+
|a                      string                                   |
|b                      int                                      |
|c                      string                                   |
|d                      string                                   |
|                                                                |
|# Partition Information                                         |
|# col_name             data_type               comment          |
|                                                                |
|c                      string                                   |
|d                      string                                   |
+----------------------------------------------------------------+
```

**Spark 2.0**
- **Before**
```scala
scala> sql("CREATE TABLE partitioned_table (a STRING, b INT) PARTITIONED BY (c STRING, d STRING)")
res0: org.apache.spark.sql.DataFrame = []

scala> sql("ALTER TABLE partitioned_table ADD PARTITION (c='Us', d=1)")
res1: org.apache.spark.sql.DataFrame = []

scala> sql("DESC partitioned_table PARTITION (c='Us', d=1)").show(false)
org.apache.spark.sql.catalyst.parser.ParseException:
Unsupported SQL statement
```

- **After**
```scala
scala> sql("CREATE TABLE partitioned_table (a STRING, b INT) PARTITIONED BY (c STRING, d STRING)")
res0: org.apache.spark.sql.DataFrame = []

scala> sql("ALTER TABLE partitioned_table ADD PARTITION (c='Us', d=1)")
res1: org.apache.spark.sql.DataFrame = []

scala> sql("DESC partitioned_table PARTITION (c='Us', d=1)").show(false)
+-----------------------+---------+-------+
|col_name               |data_type|comment|
+-----------------------+---------+-------+
|a                      |string   |null   |
|b                      |int      |null   |
|c                      |string   |null   |
|d                      |string   |null   |
|# Partition Information|         |       |
|# col_name             |data_type|comment|
|c                      |string   |null   |
|d                      |string   |null   |
+-----------------------+---------+-------+

scala> sql("DESC EXTENDED partitioned_table PARTITION (c='Us', d=1)").show(100,false)
+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+---------+-------+
|col_name                                                                                                                                                                                                                                                                                                                                                                                                                                                                           |data_type|comment|
+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+---------+-------+
|a                                                                                                                                                                                                                                                                                                                                                                                                                                                                                  |string   |null   |
|b                                                                                                                                                                                                                                                                                                                                                                                                                                                                                  |int      |null   |
|c                                                                                                                                                                                                                                                                                                                                                                                                                                                                                  |string   |null   |
|d                                                                                                                                                                                                                                                                                                                                                                                                                                                                                  |string   |null   |
|# Partition Information                                                                                                                                                                                                                                                                                                                                                                                                                                                            |         |       |
|# col_name                                                                                                                                                                                                                                                                                                                                                                                                                                                                         |data_type|comment|
|c                                                                                                                                                                                                                                                                                                                                                                                                                                                                                  |string   |null   |
|d                                                                                                                                                                                                                                                                                                                                                                                                                                                                                  |string   |null   |
|                                                                                                                                                                                                                                                                                                                                                                                                                                                                                   |         |       |
|Detailed Partition Information CatalogPartition(
        Partition Values: [Us, 1]
        Storage(Location: file:/Users/dhyun/SPARK-17612-DESC-PARTITION/spark-warehouse/partitioned_table/c=Us/d=1, InputFormat: org.apache.hadoop.mapred.TextInputFormat, OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, Serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, Properties: [serialization.format=1])
        Partition Parameters:{transient_lastDdlTime=1475001066})|         |       |
+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+---------+-------+

scala> sql("DESC FORMATTED partitioned_table PARTITION (c='Us', d=1)").show(100,false)
+--------------------------------+---------------------------------------------------------------------------------------+-------+
|col_name                        |data_type                                                                              |comment|
+--------------------------------+---------------------------------------------------------------------------------------+-------+
|a                               |string                                                                                 |null   |
|b                               |int                                                                                    |null   |
|c                               |string                                                                                 |null   |
|d                               |string                                                                                 |null   |
|# Partition Information         |                                                                                       |       |
|# col_name                      |data_type                                                                              |comment|
|c                               |string                                                                                 |null   |
|d                               |string                                                                                 |null   |
|                                |                                                                                       |       |
|# Detailed Partition Information|                                                                                       |       |
|Partition Value:                |[Us, 1]                                                                                |       |
|Database:                       |default                                                                                |       |
|Table:                          |partitioned_table                                                                      |       |
|Location:                       |file:/Users/dhyun/SPARK-17612-DESC-PARTITION/spark-warehouse/partitioned_table/c=Us/d=1|       |
|Partition Parameters:           |                                                                                       |       |
|  transient_lastDdlTime         |1475001066                                                                             |       |
|                                |                                                                                       |       |
|# Storage Information           |                                                                                       |       |
|SerDe Library:                  |org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe                                     |       |
|InputFormat:                    |org.apache.hadoop.mapred.TextInputFormat                                               |       |
|OutputFormat:                   |org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat                             |       |
|Compressed:                     |No                                                                                     |       |
|Storage Desc Parameters:        |                                                                                       |       |
|  serialization.format          |1                                                                                      |       |
+--------------------------------+---------------------------------------------------------------------------------------+-------+
```

## How was this patch tested?

Pass the Jenkins tests with a new testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15168 from dongjoon-hyun/SPARK-17612.
2016-09-29 15:30:18 -07:00
Liang-Chi Hsieh 566d7f2827 [SPARK-17653][SQL] Remove unnecessary distincts in multiple unions
## What changes were proposed in this pull request?

Currently for `Union [Distinct]`, a `Distinct` operator is necessary to be on the top of `Union`. Once there are adjacent `Union [Distinct]`,  there will be multiple `Distinct` in the query plan.

E.g.,

For a query like: select 1 a union select 2 b union select 3 c

Before this patch, its physical plan looks like:

    *HashAggregate(keys=[a#13], functions=[])
    +- Exchange hashpartitioning(a#13, 200)
       +- *HashAggregate(keys=[a#13], functions=[])
          +- Union
             :- *HashAggregate(keys=[a#13], functions=[])
             :  +- Exchange hashpartitioning(a#13, 200)
             :     +- *HashAggregate(keys=[a#13], functions=[])
             :        +- Union
             :           :- *Project [1 AS a#13]
             :           :  +- Scan OneRowRelation[]
             :           +- *Project [2 AS b#14]
             :              +- Scan OneRowRelation[]
             +- *Project [3 AS c#15]
                +- Scan OneRowRelation[]

Only the top distinct should be necessary.

After this patch, the physical plan looks like:

    *HashAggregate(keys=[a#221], functions=[], output=[a#221])
    +- Exchange hashpartitioning(a#221, 5)
       +- *HashAggregate(keys=[a#221], functions=[], output=[a#221])
          +- Union
             :- *Project [1 AS a#221]
             :  +- Scan OneRowRelation[]
             :- *Project [2 AS b#222]
             :  +- Scan OneRowRelation[]
             +- *Project [3 AS c#223]
                +- Scan OneRowRelation[]

## How was this patch tested?

Jenkins tests.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #15238 from viirya/remove-extra-distinct-union.
2016-09-29 14:30:23 -07:00
Michael Armbrust fe33121a53 [SPARK-17699] Support for parsing JSON string columns
Spark SQL has great support for reading text files that contain JSON data.  However, in many cases the JSON data is just one column amongst others.  This is particularly true when reading from sources such as Kafka.  This PR adds a new functions `from_json` that converts a string column into a nested `StructType` with a user specified schema.

Example usage:
```scala
val df = Seq("""{"a": 1}""").toDS()
val schema = new StructType().add("a", IntegerType)

df.select(from_json($"value", schema) as 'json) // => [json: <a: int>]
```

This PR adds support for java, scala and python.  I leveraged our existing JSON parsing support by moving it into catalyst (so that we could define expressions using it).  I left SQL out for now, because I'm not sure how users would specify a schema.

Author: Michael Armbrust <michael@databricks.com>

Closes #15274 from marmbrus/jsonParser.
2016-09-29 13:01:10 -07:00
Josh Rosen 37eb9184f1 [SPARK-17712][SQL] Fix invalid pushdown of data-independent filters beneath aggregates
## What changes were proposed in this pull request?

This patch fixes a minor correctness issue impacting the pushdown of filters beneath aggregates. Specifically, if a filter condition references no grouping or aggregate columns (e.g. `WHERE false`) then it would be incorrectly pushed beneath an aggregate.

Intuitively, the only case where you can push a filter beneath an aggregate is when that filter is deterministic and is defined over the grouping columns / expressions, since in that case the filter is acting to exclude entire groups from the query (like a `HAVING` clause). The existing code would only push deterministic filters beneath aggregates when all of the filter's references were grouping columns, but this logic missed the case where a filter has no references. For example, `WHERE false` is deterministic but is independent of the actual data.

This patch fixes this minor bug by adding a new check to ensure that we don't push filters beneath aggregates when those filters don't reference any columns.

## How was this patch tested?

New regression test in FilterPushdownSuite.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #15289 from JoshRosen/SPARK-17712.
2016-09-28 19:03:05 -07:00
Herman van Hovell 7d09232028 [SPARK-17641][SQL] Collect_list/Collect_set should not collect null values.
## What changes were proposed in this pull request?
We added native versions of `collect_set` and `collect_list` in Spark 2.0. These currently also (try to) collect null values, this is different from the original Hive implementation. This PR fixes this by adding a null check to the `Collect.update` method.

## How was this patch tested?
Added a regression test to `DataFrameAggregateSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #15208 from hvanhovell/SPARK-17641.
2016-09-28 16:25:10 -07:00
Josh Rosen 2f84a68660 [SPARK-17618] Guard against invalid comparisons between UnsafeRow and other formats
This patch ports changes from #15185 to Spark 2.x. In that patch, a  correctness bug in Spark 1.6.x which was caused by an invalid `equals()` comparison between an `UnsafeRow` and another row of a different format. Spark 2.x is not affected by that specific correctness bug but it can still reap the error-prevention benefits of that patch's changes, which modify  ``UnsafeRow.equals()` to throw an IllegalArgumentException if it is called with an object that is not an `UnsafeRow`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #15265 from JoshRosen/SPARK-17618-master.
2016-09-27 14:14:27 -07:00
Reynold Xin 120723f934 [SPARK-17682][SQL] Mark children as final for unary, binary, leaf expressions and plan nodes
## What changes were proposed in this pull request?
This patch marks the children method as final in unary, binary, and leaf expressions and plan nodes (both logical plan and physical plan), as brought up in http://apache-spark-developers-list.1001551.n3.nabble.com/Should-LeafExpression-have-children-final-override-like-Nondeterministic-td19104.html

## How was this patch tested?
This is a simple modifier change and has no impact on test coverage.

Author: Reynold Xin <rxin@databricks.com>

Closes #15256 from rxin/SPARK-17682.
2016-09-27 10:20:30 -07:00
Kazuaki Ishizaki 85b0a15754 [SPARK-15962][SQL] Introduce implementation with a dense format for UnsafeArrayData
## What changes were proposed in this pull request?

This PR introduces more compact representation for ```UnsafeArrayData```.

```UnsafeArrayData``` needs to accept ```null``` value in each entry of an array. In the current version, it has three parts
```
[numElements] [offsets] [values]
```
`Offsets` has the number of `numElements`, and represents `null` if its value is negative. It may increase memory footprint, and introduces an indirection for accessing each of `values`.

This PR uses bitvectors to represent nullability for each element like `UnsafeRow`, and eliminates an indirection for accessing each element. The new ```UnsafeArrayData``` has four parts.
```
[numElements][null bits][values or offset&length][variable length portion]
```
In the `null bits` region, we store 1 bit per element, represents whether an element is null. Its total size is ceil(numElements / 8) bytes, and it is aligned to 8-byte boundaries.
In the `values or offset&length` region, we store the content of elements. For fields that hold fixed-length primitive types, such as long, double, or int, we store the value directly in the field. For fields with non-primitive or variable-length values, we store a relative offset (w.r.t. the base address of the array) that points to the beginning of the variable-length field and length (they are combined into a long). Each is word-aligned. For `variable length portion`, each is aligned to 8-byte boundaries.

The new format can reduce memory footprint and improve performance of accessing each element. An example of memory foot comparison:
1024x1024 elements integer array
Size of ```baseObject``` for ```UnsafeArrayData```: 8 + 1024x1024 + 1024x1024 = 2M bytes
Size of ```baseObject``` for ```UnsafeArrayData```: 8 + 1024x1024/8 + 1024x1024 = 1.25M bytes

In summary, we got 1.0-2.6x performance improvements over the code before applying this PR.
Here are performance results of [benchmark programs](04d2e4b6db/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala):

**Read UnsafeArrayData**: 1.7x and 1.6x performance improvements over the code before applying this PR
````
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)

Without SPARK-15962
Read UnsafeArrayData:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            430 /  436        390.0           2.6       1.0X
Double                                         456 /  485        367.8           2.7       0.9X

With SPARK-15962
Read UnsafeArrayData:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            252 /  260        666.1           1.5       1.0X
Double                                         281 /  292        597.7           1.7       0.9X
````
**Write UnsafeArrayData**: 1.0x and 1.1x performance improvements over the code before applying this PR
````
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)

Without SPARK-15962
Write UnsafeArrayData:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            203 /  273        103.4           9.7       1.0X
Double                                         239 /  356         87.9          11.4       0.8X

With SPARK-15962
Write UnsafeArrayData:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            196 /  249        107.0           9.3       1.0X
Double                                         227 /  367         92.3          10.8       0.9X
````

**Get primitive array from UnsafeArrayData**: 2.6x and 1.6x performance improvements over the code before applying this PR
````
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)

Without SPARK-15962
Get primitive array from UnsafeArrayData: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            207 /  217        304.2           3.3       1.0X
Double                                         257 /  363        245.2           4.1       0.8X

With SPARK-15962
Get primitive array from UnsafeArrayData: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            151 /  198        415.8           2.4       1.0X
Double                                         214 /  394        293.6           3.4       0.7X
````

**Create UnsafeArrayData from primitive array**: 1.7x and 2.1x performance improvements over the code before applying this PR
````
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)

Without SPARK-15962
Create UnsafeArrayData from primitive array: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            340 /  385        185.1           5.4       1.0X
Double                                         479 /  705        131.3           7.6       0.7X

With SPARK-15962
Create UnsafeArrayData from primitive array: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            206 /  211        306.0           3.3       1.0X
Double                                         232 /  406        271.6           3.7       0.9X
````

1.7x and 1.4x performance improvements in [```UDTSerializationBenchmark```](https://github.com/apache/spark/blob/master/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala)  over the code before applying this PR
````
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)

Without SPARK-15962
VectorUDT de/serialization:              Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
serialize                                      442 /  533          0.0      441927.1       1.0X
deserialize                                    217 /  274          0.0      217087.6       2.0X

With SPARK-15962
VectorUDT de/serialization:              Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
serialize                                      265 /  318          0.0      265138.5       1.0X
deserialize                                    155 /  197          0.0      154611.4       1.7X
````

## How was this patch tested?

Added unit tests into ```UnsafeArraySuite```

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #13680 from kiszk/SPARK-15962.
2016-09-27 14:18:32 +08:00
xin wu de333d121d [SPARK-17551][SQL] Add DataFrame API for null ordering
## What changes were proposed in this pull request?
This pull request adds Scala/Java DataFrame API for null ordering (NULLS FIRST | LAST).

Also did some minor clean up for related code (e.g. incorrect indentation), and renamed "orderby-nulls-ordering.sql" to be consistent with existing test files.

## How was this patch tested?
Added a new test case in DataFrameSuite.

Author: petermaxlee <petermaxlee@gmail.com>
Author: Xin Wu <xinwu@us.ibm.com>

Closes #15123 from petermaxlee/SPARK-17551.
2016-09-25 16:46:12 -07:00
Herman van Hovell 0d63487502 [SPARK-17616][SQL] Support a single distinct aggregate combined with a non-partial aggregate
## What changes were proposed in this pull request?
We currently cannot execute an aggregate that contains a single distinct aggregate function and an one or more non-partially plannable aggregate functions, for example:
```sql
select   grp,
         collect_list(col1),
         count(distinct col2)
from     tbl_a
group by 1
```
This is a regression from Spark 1.6. This is caused by the fact that the single distinct aggregation code path assumes that all aggregates can be planned in two phases (is partially aggregatable). This PR works around this issue by triggering the `RewriteDistinctAggregates` in such cases (this is similar to the approach taken in 1.6).

## How was this patch tested?
Created `RewriteDistinctAggregatesSuite` which checks if the aggregates with distinct aggregate functions get rewritten into two `Aggregates` and an `Expand`. Added a regression test to `DataFrameAggregateSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #15187 from hvanhovell/SPARK-17616.
2016-09-22 14:29:27 -07:00
Wenchen Fan b50b34f561 [SPARK-17609][SQL] SessionCatalog.tableExists should not check temp view
## What changes were proposed in this pull request?

After #15054 , there is no place in Spark SQL that need `SessionCatalog.tableExists` to check temp views, so this PR makes `SessionCatalog.tableExists` only check permanent table/view and removes some hacks.

This PR also improves the `getTempViewOrPermanentTableMetadata` that is introduced in  #15054 , to make the code simpler.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15160 from cloud-fan/exists.
2016-09-22 12:52:09 +08:00
Davies Liu 8bde03bf9a [SPARK-17494][SQL] changePrecision() on compact decimal should respect rounding mode
## What changes were proposed in this pull request?

Floor()/Ceil() of decimal is implemented using changePrecision() by passing a rounding mode, but the rounding mode is not respected when the decimal is in compact mode (could fit within a Long).

This Update the changePrecision() to respect rounding mode, which could be ROUND_FLOOR, ROUND_CEIL, ROUND_HALF_UP, ROUND_HALF_EVEN.

## How was this patch tested?

Added regression tests.

Author: Davies Liu <davies@databricks.com>

Closes #15154 from davies/decimal_round.
2016-09-21 21:02:30 -07:00
Liang-Chi Hsieh 248922fd4f [SPARK-17590][SQL] Analyze CTE definitions at once and allow CTE subquery to define CTE
## What changes were proposed in this pull request?

We substitute logical plan with CTE definitions in the analyzer rule CTESubstitution. A CTE definition can be used in the logical plan for multiple times, and its analyzed logical plan should be the same. We should not analyze CTE definitions multiple times when they are reused in the query.

By analyzing CTE definitions before substitution, we can support defining CTE in subquery.

## How was this patch tested?

Jenkins tests.

Author: Liang-Chi Hsieh <simonh@tw.ibm.com>

Closes #15146 from viirya/cte-analysis-once.
2016-09-21 06:53:42 -07:00
Sean Zhong 3977223a32 [SPARK-17617][SQL] Remainder(%) expression.eval returns incorrect result on double value
## What changes were proposed in this pull request?

Remainder(%) expression's `eval()` returns incorrect result when the dividend is a big double. The reason is that Remainder converts the double dividend to decimal to do "%", and that lose precision.

This bug only affects the `eval()` that is used by constant folding, the codegen path is not impacted.

### Before change
```
scala> -5083676433652386516D % 10
res2: Double = -6.0

scala> spark.sql("select -5083676433652386516D % 10 as a").show
+---+
|  a|
+---+
|0.0|
+---+
```

### After change
```
scala> spark.sql("select -5083676433652386516D % 10 as a").show
+----+
|   a|
+----+
|-6.0|
+----+
```

## How was this patch tested?

Unit test.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #15171 from clockfly/SPARK-17617.
2016-09-21 16:53:34 +08:00
gatorsmile d5ec5dbb0d [SPARK-17502][SQL] Fix Multiple Bugs in DDL Statements on Temporary Views
### What changes were proposed in this pull request?
- When the permanent tables/views do not exist but the temporary view exists, the expected error should be `NoSuchTableException` for partition-related ALTER TABLE commands. However, it always reports a confusing error message. For example,
```
Partition spec is invalid. The spec (a, b) must match the partition spec () defined in table '`testview`';
```
- When the permanent tables/views do not exist but the temporary view exists, the expected error should be `NoSuchTableException` for `ALTER TABLE ... UNSET TBLPROPERTIES`. However, it reports a missing table property. For example,
```
Attempted to unset non-existent property 'p' in table '`testView`';
```
- When `ANALYZE TABLE` is called on a view or a temporary view, we should issue an error message. However, it reports a strange error:
```
ANALYZE TABLE is not supported for Project
```

- When inserting into a temporary view that is generated from `Range`, we will get the following error message:
```
assertion failed: No plan for 'InsertIntoTable Range (0, 10, step=1, splits=Some(1)), false, false
+- Project [1 AS 1#20]
   +- OneRowRelation$
```

This PR is to fix the above four issues.

### How was this patch tested?
Added multiple test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15054 from gatorsmile/tempViewDDL.
2016-09-20 20:11:48 +08:00
Josh Rosen e719b1c045 [SPARK-17160] Properly escape field names in code-generated error messages
This patch addresses a corner-case escaping bug where field names which contain special characters were unsafely interpolated into error message string literals in generated Java code, leading to compilation errors.

This patch addresses these issues by using `addReferenceObj` to store the error messages as string fields rather than inline string constants.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #15156 from JoshRosen/SPARK-17160.
2016-09-19 20:20:36 -07:00
Davies Liu d8104158a9 [SPARK-17100] [SQL] fix Python udf in filter on top of outer join
## What changes were proposed in this pull request?

In optimizer, we try to evaluate the condition to see whether it's nullable or not, but some expressions are not evaluable, we should check that before evaluate it.

## How was this patch tested?

Added regression tests.

Author: Davies Liu <davies@databricks.com>

Closes #15103 from davies/udf_join.
2016-09-19 13:24:16 -07:00
jiangxingbo 5d3f4615f8
[SPARK-17506][SQL] Improve the check double values equality rule.
## What changes were proposed in this pull request?

In `ExpressionEvalHelper`, we check the equality between two double values by comparing whether the expected value is within the range [target - tolerance, target + tolerance], but this can cause a negative false when the compared numerics are very large.
Before:
```
val1 = 1.6358558070241E306
val2 = 1.6358558070240974E306
ExpressionEvalHelper.compareResults(val1, val2)
false
```
In fact, `val1` and `val2` are but with different precisions, we should tolerant this case by comparing with percentage range, eg.,expected is within range [target - target * tolerance_percentage, target + target * tolerance_percentage].
After:
```
val1 = 1.6358558070241E306
val2 = 1.6358558070240974E306
ExpressionEvalHelper.compareResults(val1, val2)
true
```

## How was this patch tested?

Exsiting testcases.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15059 from jiangxb1987/deq.
2016-09-18 16:04:37 +01:00
Wenchen Fan 3fe630d314 [SPARK-17541][SQL] fix some DDL bugs about table management when same-name temp view exists
## What changes were proposed in this pull request?

In `SessionCatalog`, we have several operations(`tableExists`, `dropTable`, `loopupRelation`, etc) that handle both temp views and metastore tables/views. This brings some bugs to DDL commands that want to handle temp view only or metastore table/view only. These bugs are:

1. `CREATE TABLE USING` will fail if a same-name temp view exists
2. `Catalog.dropTempView`will un-cache and drop metastore table if a same-name table exists
3. `saveAsTable` will fail or have unexpected behaviour if a same-name temp view exists.

These bug fixes are pulled out from https://github.com/apache/spark/pull/14962 and targets both master and 2.0 branch

## How was this patch tested?

new regression tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15099 from cloud-fan/fix-view.
2016-09-18 21:15:35 +08:00
hyukjinkwon 86c2d393a5
[SPARK-17480][SQL][FOLLOWUP] Fix more instances which calls List.length/size which is O(n)
## What changes were proposed in this pull request?

This PR fixes all the instances which was fixed in the previous PR.

To make sure, I manually debugged and also checked the Scala source. `length` in [LinearSeqOptimized.scala#L49-L57](https://github.com/scala/scala/blob/2.11.x/src/library/scala/collection/LinearSeqOptimized.scala#L49-L57) is O(n). Also, `size` calls `length` via [SeqLike.scala#L106](https://github.com/scala/scala/blob/2.11.x/src/library/scala/collection/SeqLike.scala#L106).

For debugging, I have created these as below:

```scala
ArrayBuffer(1, 2, 3)
Array(1, 2, 3)
List(1, 2, 3)
Seq(1, 2, 3)
```

and then called `size` and `length` for each to debug.

## How was this patch tested?

I ran the bash as below on Mac

```bash
find . -name *.scala -type f -exec grep -il "while (.*\\.length)" {} \; | grep "src/main"
find . -name *.scala -type f -exec grep -il "while (.*\\.size)" {} \; | grep "src/main"
```

and then checked each.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15093 from HyukjinKwon/SPARK-17480-followup.
2016-09-17 16:52:30 +01:00
Marcelo Vanzin 39e2bad6a8 [SPARK-17549][SQL] Only collect table size stat in driver for cached relation.
The existing code caches all stats for all columns for each partition
in the driver; for a large relation, this causes extreme memory usage,
which leads to gc hell and application failures.

It seems that only the size in bytes of the data is actually used in the
driver, so instead just colllect that. In executors, the full stats are
still kept, but that's not a big problem; we expect the data to be distributed
and thus not really incur in too much memory pressure in each individual
executor.

There are also potential improvements on the executor side, since the data
being stored currently is very wasteful (e.g. storing boxed types vs.
primitive types for stats). But that's a separate issue.

On a mildly related change, I'm also adding code to catch exceptions in the
code generator since Janino was breaking with the test data I tried this
patch on.

Tested with unit tests and by doing a count a very wide table (20k columns)
with many partitions.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #15112 from vanzin/SPARK-17549.
2016-09-16 14:02:56 -07:00
Sean Zhong a425a37a5d [SPARK-17426][SQL] Refactor TreeNode.toJSON to avoid OOM when converting unknown fields to JSON
## What changes were proposed in this pull request?

This PR is a follow up of SPARK-17356. Current implementation of `TreeNode.toJSON` recursively converts all fields of TreeNode to JSON, even if the field is of type `Seq` or type Map. This may trigger out of memory exception in cases like:

1. the Seq or Map can be very big. Converting them to JSON may take huge memory, which may trigger out of memory error.
2. Some user space input may also be propagated to the Plan. The user space input can be of arbitrary type, and may also be self-referencing. Trying to print user space input to JSON may trigger out of memory error or stack overflow error.

For a code example, please check the Jira description of SPARK-17426.

In this PR, we refactor the `TreeNode.toJSON` so that we only convert a field to JSON string if the field is a safe type.

## How was this patch tested?

Unit test.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #14990 from clockfly/json_oom2.
2016-09-16 19:37:30 +08:00
Andrew Ray b72486f82d [SPARK-17458][SQL] Alias specified for aggregates in a pivot are not honored
## What changes were proposed in this pull request?

This change preserves aliases that are given for pivot aggregations

## How was this patch tested?

New unit test

Author: Andrew Ray <ray.andrew@gmail.com>

Closes #15111 from aray/SPARK-17458.
2016-09-15 21:45:29 +02:00
Sean Zhong a6b8182006 [SPARK-17364][SQL] Antlr lexer wrongly treats full qualified identifier as a decimal number token when parsing SQL string
## What changes were proposed in this pull request?

The Antlr lexer we use to tokenize a SQL string may wrongly tokenize a fully qualified identifier as a decimal number token. For example, table identifier `default.123_table` is wrongly tokenized as
```
default // Matches lexer rule IDENTIFIER
.123 // Matches lexer rule DECIMAL_VALUE
_TABLE // Matches lexer rule IDENTIFIER
```

The correct tokenization for `default.123_table` should be:
```
default // Matches lexer rule IDENTIFIER,
. // Matches a single dot
123_TABLE // Matches lexer rule IDENTIFIER
```

This PR fix the Antlr grammar so that it can tokenize fully qualified identifier correctly:
1. Fully qualified table name can be parsed correctly. For example, `select * from database.123_suffix`.
2. Fully qualified column name can be parsed correctly, for example `select a.123_suffix from a`.

### Before change

#### Case 1: Failed to parse fully qualified column name

```
scala> spark.sql("select a.123_column from a").show
org.apache.spark.sql.catalyst.parser.ParseException:
extraneous input '.123' expecting {<EOF>,
...
, IDENTIFIER, BACKQUOTED_IDENTIFIER}(line 1, pos 8)
== SQL ==
select a.123_column from a
--------^^^
```

#### Case 2: Failed to parse fully qualified table name
```
scala> spark.sql("select * from default.123_table")
org.apache.spark.sql.catalyst.parser.ParseException:
extraneous input '.123' expecting {<EOF>,
...
IDENTIFIER, BACKQUOTED_IDENTIFIER}(line 1, pos 21)

== SQL ==
select * from default.123_table
---------------------^^^
```

### After Change

#### Case 1: fully qualified column name, no ParseException thrown
```
scala> spark.sql("select a.123_column from a").show
```

#### Case 2: fully qualified table name, no ParseException thrown
```
scala> spark.sql("select * from default.123_table")
```

## How was this patch tested?

Unit test.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #15006 from clockfly/SPARK-17364.
2016-09-15 20:53:48 +02:00
岑玉海 fe767395ff [SPARK-17429][SQL] use ImplicitCastInputTypes with function Length
## What changes were proposed in this pull request?
select length(11);
select length(2.0);
these sql will return errors, but hive is ok.
this PR will support casting input types implicitly for function length
the correct result is:
select length(11) return 2
select length(2.0) return 3

Author: 岑玉海 <261810726@qq.com>
Author: cenyuhai <cenyuhai@didichuxing.com>

Closes #15014 from cenyuhai/SPARK-17429.
2016-09-15 20:45:00 +02:00
Herman van Hovell d403562eb4 [SPARK-17114][SQL] Fix aggregates grouped by literals with empty input
## What changes were proposed in this pull request?
This PR fixes an issue with aggregates that have an empty input, and use a literals as their grouping keys. These aggregates are currently interpreted as aggregates **without** grouping keys, this triggers the ungrouped code path (which aways returns a single row).

This PR fixes the `RemoveLiteralFromGroupExpressions` optimizer rule, which changes the semantics of the Aggregate by eliminating all literal grouping keys.

## How was this patch tested?
Added tests to `SQLQueryTestSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #15101 from hvanhovell/SPARK-17114-3.
2016-09-15 20:24:15 +02:00
Adam Roberts f893e26250 [SPARK-17524][TESTS] Use specified spark.buffer.pageSize
## What changes were proposed in this pull request?

This PR has the appendRowUntilExceedingPageSize test in RowBasedKeyValueBatchSuite use whatever spark.buffer.pageSize value a user has specified to prevent a test failure for anyone testing Apache Spark on a box with a reduced page size. The test is currently hardcoded to use the default page size which is 64 MB so this minor PR is a test improvement

## How was this patch tested?
Existing unit tests with 1 MB page size and with 64 MB (the default) page size

Author: Adam Roberts <aroberts@uk.ibm.com>

Closes #15079 from a-roberts/patch-5.
2016-09-15 09:37:12 +01:00
Xin Wu 040e46979d [SPARK-10747][SQL] Support NULLS FIRST|LAST clause in ORDER BY
## What changes were proposed in this pull request?
Currently, ORDER BY clause returns nulls value according to sorting order (ASC|DESC), considering null value is always smaller than non-null values.
However, SQL2003 standard support NULLS FIRST or NULLS LAST to allow users to specify whether null values should be returned first or last, regardless of sorting order (ASC|DESC).

This PR is to support this new feature.

## How was this patch tested?
New test cases are added to test NULLS FIRST|LAST for regular select queries and windowing queries.

(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Author: Xin Wu <xinwu@us.ibm.com>

Closes #14842 from xwu0226/SPARK-10747.
2016-09-14 21:14:29 +02:00
gatorsmile 52738d4e09 [SPARK-17409][SQL] Do Not Optimize Query in CTAS More Than Once
### What changes were proposed in this pull request?
As explained in https://github.com/apache/spark/pull/14797:
>Some analyzer rules have assumptions on logical plans, optimizer may break these assumption, we should not pass an optimized query plan into QueryExecution (will be analyzed again), otherwise we may some weird bugs.
For example, we have a rule for decimal calculation to promote the precision before binary operations, use PromotePrecision as placeholder to indicate that this rule should not apply twice. But a Optimizer rule will remove this placeholder, that break the assumption, then the rule applied twice, cause wrong result.

We should not optimize the query in CTAS more than once. For example,
```Scala
spark.range(99, 101).createOrReplaceTempView("tab1")
val sqlStmt = "SELECT id, cast(id as long) * cast('1.0' as decimal(38, 18)) as num FROM tab1"
sql(s"CREATE TABLE tab2 USING PARQUET AS $sqlStmt")
checkAnswer(spark.table("tab2"), sql(sqlStmt))
```
Before this PR, the results do not match
```
== Results ==
!== Correct Answer - 2 ==       == Spark Answer - 2 ==
![100,100.000000000000000000]   [100,null]
 [99,99.000000000000000000]     [99,99.000000000000000000]
```
After this PR, the results match.
```
+---+----------------------+
|id |num                   |
+---+----------------------+
|99 |99.000000000000000000 |
|100|100.000000000000000000|
+---+----------------------+
```

In this PR, we do not treat the `query` in CTAS as a child. Thus, the `query` will not be optimized when optimizing CTAS statement. However, we still need to analyze it for normalizing and verifying the CTAS in the Analyzer. Thus, we do it in the analyzer rule `PreprocessDDL`, because so far only this rule needs the analyzed plan of the `query`.

### How was this patch tested?
Added a test

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15048 from gatorsmile/ctasOptimized.
2016-09-14 23:10:20 +08:00
gatorsmile 37b93f54e8 [SPARK-17530][SQL] Add Statistics into DESCRIBE FORMATTED
### What changes were proposed in this pull request?
Statistics is missing in the output of `DESCRIBE FORMATTED`. This PR is to add it. After the PR, the output will be like:
```
+----------------------------+----------------------------------------------------------------------------------------------------------------------+-------+
|col_name                    |data_type                                                                                                             |comment|
+----------------------------+----------------------------------------------------------------------------------------------------------------------+-------+
|key                         |string                                                                                                                |null   |
|value                       |string                                                                                                                |null   |
|                            |                                                                                                                      |       |
|# Detailed Table Information|                                                                                                                      |       |
|Database:                   |default                                                                                                               |       |
|Owner:                      |xiaoli                                                                                                                |       |
|Create Time:                |Tue Sep 13 14:36:57 PDT 2016                                                                                          |       |
|Last Access Time:           |Wed Dec 31 16:00:00 PST 1969                                                                                          |       |
|Location:                   |file:/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/warehouse-9982e1db-df17-4376-a140-dbbee0203d83/texttable|       |
|Table Type:                 |MANAGED                                                                                                               |       |
|Statistics:                 |sizeInBytes=5812, rowCount=500, isBroadcastable=false                                                                 |       |
|Table Parameters:           |                                                                                                                      |       |
|  rawDataSize               |-1                                                                                                                    |       |
|  numFiles                  |1                                                                                                                     |       |
|  transient_lastDdlTime     |1473802620                                                                                                            |       |
|  totalSize                 |5812                                                                                                                  |       |
|  COLUMN_STATS_ACCURATE     |false                                                                                                                 |       |
|  numRows                   |-1                                                                                                                    |       |
|                            |                                                                                                                      |       |
|# Storage Information       |                                                                                                                      |       |
|SerDe Library:              |org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe                                                                    |       |
|InputFormat:                |org.apache.hadoop.mapred.TextInputFormat                                                                              |       |
|OutputFormat:               |org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat                                                            |       |
|Compressed:                 |No                                                                                                                    |       |
|Storage Desc Parameters:    |                                                                                                                      |       |
|  serialization.format      |1                                                                                                                     |       |
+----------------------------+----------------------------------------------------------------------------------------------------------------------+-------+
```

Also improve the output of statistics in `DESCRIBE EXTENDED` by removing duplicate `Statistics`. Below is the example after the PR:

```
+----------------------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-------+
|col_name                    |data_type                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                        |comment|
+----------------------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-------+
|key                         |string                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                           |null   |
|value                       |string                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                           |null   |
|                            |                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                 |       |
|# Detailed Table Information|CatalogTable(
	Table: `default`.`texttable`
	Owner: xiaoli
	Created: Tue Sep 13 14:38:43 PDT 2016
	Last Access: Wed Dec 31 16:00:00 PST 1969
	Type: MANAGED
	Schema: [StructField(key,StringType,true), StructField(value,StringType,true)]
	Provider: hive
	Properties: [rawDataSize=-1, numFiles=1, transient_lastDdlTime=1473802726, totalSize=5812, COLUMN_STATS_ACCURATE=false, numRows=-1]
	Statistics: sizeInBytes=5812, rowCount=500, isBroadcastable=false
	Storage(Location: file:/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/warehouse-8ea5c5a0-5680-4778-91cb-c6334cf8a708/texttable, InputFormat: org.apache.hadoop.mapred.TextInputFormat, OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, Serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, Properties: [serialization.format=1]))|       |
+----------------------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-------+
```

### How was this patch tested?
Manually tested.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15083 from gatorsmile/descFormattedStats.
2016-09-14 00:37:42 +02:00
jiangxingbo 4ba63b193c [SPARK-17142][SQL] Complex query triggers binding error in HashAggregateExec
## What changes were proposed in this pull request?

In `ReorderAssociativeOperator` rule, we extract foldable expressions with Add/Multiply arithmetics, and replace with eval literal. For example, `(a + 1) + (b + 2)` is optimized to `(a + b + 3)` by this rule.
For aggregate operator, output expressions should be derived from groupingExpressions, current implemenation of `ReorderAssociativeOperator` rule may break this promise. A instance could be:
```
SELECT
  ((t1.a + 1) + (t2.a + 2)) AS out_col
FROM
  testdata2 AS t1
INNER JOIN
  testdata2 AS t2
ON
  (t1.a = t2.a)
GROUP BY (t1.a + 1), (t2.a + 2)
```
`((t1.a + 1) + (t2.a + 2))` is optimized to `(t1.a + t2.a + 3)`, which could not be derived from `ExpressionSet((t1.a +1), (t2.a + 2))`.
Maybe we should improve the rule of `ReorderAssociativeOperator` by adding a GroupingExpressionSet to keep Aggregate.groupingExpressions, and respect these expressions during the optimize stage.

## How was this patch tested?

Add new test case in `ReorderAssociativeOperatorSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #14917 from jiangxb1987/rao.
2016-09-13 17:04:51 +02:00
Timothy Hunter 180796ecb3 [SPARK-17439][SQL] Fixing compression issues with approximate quantiles and adding more tests
## What changes were proposed in this pull request?

This PR build on #14976 and fixes a correctness bug that would cause the wrong quantile to be returned for small target errors.

## How was this patch tested?

This PR adds 8 unit tests that were failing without the fix.

Author: Timothy Hunter <timhunter@databricks.com>
Author: Sean Owen <sowen@cloudera.com>

Closes #15002 from thunterdb/ml-1783.
2016-09-11 08:03:45 +01:00
Eric Liang 722afbb2b3 [SPARK-17405] RowBasedKeyValueBatch should use default page size to prevent OOMs
## What changes were proposed in this pull request?

Before this change, we would always allocate 64MB per aggregation task for the first-level hash map storage, even when running in low-memory situations such as local mode. This changes it to use the memory manager default page size, which is automatically reduced from 64MB in these situations.

cc ooq JoshRosen

## How was this patch tested?

Tested manually with `bin/spark-shell --master=local[32]` and verifying that `(1 to math.pow(10, 3).toInt).toDF("n").withColumn("m", 'n % 2).groupBy('m).agg(sum('n)).show` does not crash.

Author: Eric Liang <ekl@databricks.com>

Closes #15016 from ericl/sc-4483.
2016-09-08 16:47:18 -07:00
Srinivasa Reddy Vundela 76ad89e924 [MINOR][SQL] Fixing the typo in unit test
## What changes were proposed in this pull request?

Fixing the typo in the unit test of CodeGenerationSuite.scala

## How was this patch tested?
Ran the unit test after fixing the typo and it passes

Author: Srinivasa Reddy Vundela <vsr@cloudera.com>

Closes #14989 from vundela/typo_fix.
2016-09-07 12:41:03 +01:00
Daoyuan Wang 6f4aeccf8c [SPARK-17427][SQL] function SIZE should return -1 when parameter is null
## What changes were proposed in this pull request?

`select size(null)` returns -1 in Hive. In order to be compatible, we should return `-1`.

## How was this patch tested?

unit test in `CollectionFunctionsSuite` and `DataFrameFunctionsSuite`.

Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #14991 from adrian-wang/size.
2016-09-07 13:01:27 +02:00
Liwei Lin 3ce3a282c8 [SPARK-17359][SQL][MLLIB] Use ArrayBuffer.+=(A) instead of ArrayBuffer.append(A) in performance critical paths
## What changes were proposed in this pull request?

We should generally use `ArrayBuffer.+=(A)` rather than `ArrayBuffer.append(A)`, because `append(A)` would involve extra boxing / unboxing.

## How was this patch tested?

N/A

Author: Liwei Lin <lwlin7@gmail.com>

Closes #14914 from lw-lin/append_to_plus_eq_v2.
2016-09-07 10:04:00 +01:00
Herman van Hovell 4f769b903b [SPARK-17296][SQL] Simplify parser join processing.
## What changes were proposed in this pull request?
Join processing in the parser relies on the fact that the grammar produces a right nested trees, for instance the parse tree for `select * from a join b join c` is expected to produce a tree similar to `JOIN(a, JOIN(b, c))`. However there are cases in which this (invariant) is violated, like:
```sql
SELECT COUNT(1)
FROM test T1
     CROSS JOIN test T2
     JOIN test T3
      ON T3.col = T1.col
     JOIN test T4
      ON T4.col = T1.col
```
In this case the parser returns a tree in which Joins are located on both the left and the right sides of the parent join node.

This PR introduces a different grammar rule which does not make this assumption. The new rule takes a relation and searches for zero or more joined relations. As a bonus processing is much easier.

## How was this patch tested?
Existing tests and I have added a regression test to the plan parser suite.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #14867 from hvanhovell/SPARK-17296.
2016-09-07 00:44:07 +02:00
Sean Zhong 6f13aa7dfe [SPARK-17356][SQL] Fix out of memory issue when generating JSON for TreeNode
## What changes were proposed in this pull request?

class `org.apache.spark.sql.types.Metadata` is widely used in mllib to store some ml attributes. `Metadata` is commonly stored in `Alias` expression.

```
case class Alias(child: Expression, name: String)(
    val exprId: ExprId = NamedExpression.newExprId,
    val qualifier: Option[String] = None,
    val explicitMetadata: Option[Metadata] = None,
    override val isGenerated: java.lang.Boolean = false)
```

The `Metadata` can take a big memory footprint since the number of attributes is big ( in scale of million). When `toJSON` is called on `Alias` expression, the `Metadata` will also be converted to a big JSON string.
If a plan contains many such kind of `Alias` expressions, it may trigger out of memory error when `toJSON` is called, since converting all `Metadata` references to JSON will take huge memory.

With this PR, we will skip scanning Metadata when doing JSON conversion. For a reproducer of the OOM, and analysis, please look at jira https://issues.apache.org/jira/browse/SPARK-17356.

## How was this patch tested?

Existing tests.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #14915 from clockfly/json_oom.
2016-09-06 16:05:50 +08:00
Wenchen Fan c0ae6bc6ea [SPARK-17361][SQL] file-based external table without path should not be created
## What changes were proposed in this pull request?

Using the public `Catalog` API, users can create a file-based data source table, without giving the path options. For this case, currently we can create the table successfully, but fail when we read it. Ideally we should fail during creation.

This is because when we create data source table, we resolve the data source relation without validating path: `resolveRelation(checkPathExist = false)`.

Looking back to why we add this trick(`checkPathExist`), it's because when we call `resolveRelation` for managed table, we add the path to data source options but the path is not created yet. So why we add this not-yet-created path to data source options? This PR fix the problem by adding path to options after we call `resolveRelation`. Then we can remove the `checkPathExist` parameter in `DataSource.resolveRelation` and do some related cleanups.

## How was this patch tested?

existing tests and new test in `CatalogSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14921 from cloud-fan/check-path.
2016-09-06 14:17:47 +08:00
Wenchen Fan 8d08f43d09 [SPARK-17279][SQL] better error message for exceptions during ScalaUDF execution
## What changes were proposed in this pull request?

If `ScalaUDF` throws exceptions during executing user code, sometimes it's hard for users to figure out what's wrong, especially when they use Spark shell. An example
```
org.apache.spark.SparkException: Job aborted due to stage failure: Task 12 in stage 325.0 failed 4 times, most recent failure: Lost task 12.3 in stage 325.0 (TID 35622, 10.0.207.202): java.lang.NullPointerException
	at line8414e872fb8b42aba390efc153d1611a12.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$2.apply(<console>:40)
	at line8414e872fb8b42aba390efc153d1611a12.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$2.apply(<console>:40)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source)
...
```
We should catch these exceptions and rethrow them with better error message, to say that the exception is happened in scala udf.

This PR also does some clean up for `ScalaUDF` and add a unit test suite for it.

## How was this patch tested?

the new test suite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14850 from cloud-fan/npe.
2016-09-06 10:36:00 +08:00
wangzhenhua 6d86403d8b [SPARK-17072][SQL] support table-level statistics generation and storing into/loading from metastore
## What changes were proposed in this pull request?

1. Support generation table-level statistics for
    - hive tables in HiveExternalCatalog
    - data source tables in HiveExternalCatalog
    - data source tables in InMemoryCatalog.
2. Add a property "catalogStats" in CatalogTable to hold statistics in Spark side.
3. Put logics of statistics transformation between Spark and Hive in HiveClientImpl.
4. Extend Statistics class by adding rowCount (will add estimatedSize when we have column stats).

## How was this patch tested?

add unit tests

Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #14712 from wzhfy/tableStats.
2016-09-05 17:32:31 +02:00
Wenchen Fan 3ccb23e445 [SPARK-17394][SQL] should not allow specify database in table/view name after RENAME TO
## What changes were proposed in this pull request?

It's really weird that we allow users to specify database in both from table name and to table name
 in `ALTER TABLE RENAME TO`, while logically we can't support rename a table to a different database.

Both postgres and MySQL disallow this syntax, it's reasonable to follow them and simply our code.

## How was this patch tested?

new test in `DDLCommandSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14955 from cloud-fan/rename.
2016-09-05 13:09:20 +08:00
Shivansh e75c162e9e [SPARK-17308] Improved the spark core code by replacing all pattern match on boolean value by if/else block.
## What changes were proposed in this pull request?
Improved the code quality of spark by replacing all pattern match on boolean value by if/else block.

## How was this patch tested?

By running the tests

Author: Shivansh <shiv4nsh@gmail.com>

Closes #14873 from shiv4nsh/SPARK-17308.
2016-09-04 12:39:26 +01:00
gatorsmile 6b156e2fcf [SPARK-17324][SQL] Remove Direct Usage of HiveClient in InsertIntoHiveTable
### What changes were proposed in this pull request?
This is another step to get rid of HiveClient from `HiveSessionState`. All the metastore interactions should be through `ExternalCatalog` interface. However, the existing implementation of `InsertIntoHiveTable ` still requires Hive clients. This PR is to remove HiveClient by moving the metastore interactions into `ExternalCatalog`.

### How was this patch tested?
Existing test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14888 from gatorsmile/removeClientFromInsertIntoHiveTable.
2016-09-04 15:04:33 +08:00
Herman van Hovell c2a1576c23 [SPARK-17335][SQL] Fix ArrayType and MapType CatalogString.
## What changes were proposed in this pull request?
the `catalogString` for `ArrayType` and `MapType` currently calls the `simpleString` method on its children. This is a problem when the child is a struct, the `struct.simpleString` implementation truncates the number of fields it shows (25 at max). This breaks the generation of a proper `catalogString`, and has shown to cause errors while writing to Hive.

This PR fixes this by providing proper `catalogString` implementations for `ArrayData` or `MapData`.

## How was this patch tested?
Added testing for `catalogString` to `DataTypeSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #14938 from hvanhovell/SPARK-17335.
2016-09-03 19:02:20 +02:00
Srinath Shankar e6132a6cf1 [SPARK-17298][SQL] Require explicit CROSS join for cartesian products
## What changes were proposed in this pull request?

Require the use of CROSS join syntax in SQL (and a new crossJoin
DataFrame API) to specify explicit cartesian products between relations.
By cartesian product we mean a join between relations R and S where
there is no join condition involving columns from both R and S.

If a cartesian product is detected in the absence of an explicit CROSS
join, an error must be thrown. Turning on the
"spark.sql.crossJoin.enabled" configuration flag will disable this check
and allow cartesian products without an explicit CROSS join.

The new crossJoin DataFrame API must be used to specify explicit cross
joins. The existing join(DataFrame) method will produce a INNER join
that will require a subsequent join condition.
That is df1.join(df2) is equivalent to select * from df1, df2.

## How was this patch tested?

Added cross-join.sql to the SQLQueryTestSuite to test the check for cartesian products. Added a couple of tests to the DataFrameJoinSuite to test the crossJoin API. Modified various other test suites to explicitly specify a cross join where an INNER join or a comma-separated list was previously used.

Author: Srinath Shankar <srinath@databricks.com>

Closes #14866 from srinathshankar/crossjoin.
2016-09-03 00:20:43 +02:00
gatorsmile 247a4faf06 [SPARK-16935][SQL] Verification of Function-related ExternalCatalog APIs
### What changes were proposed in this pull request?
Function-related `HiveExternalCatalog` APIs do not have enough verification logics. After the PR, `HiveExternalCatalog` and `InMemoryCatalog` become consistent in the error handling.

For example, below is the exception we got when calling `renameFunction`.
```
15:13:40.369 WARN org.apache.hadoop.hive.metastore.ObjectStore: Failed to get database db1, returning NoSuchObjectException
15:13:40.377 WARN org.apache.hadoop.hive.metastore.ObjectStore: Failed to get database db2, returning NoSuchObjectException
15:13:40.739 ERROR DataNucleus.Datastore.Persist: Update of object "org.apache.hadoop.hive.metastore.model.MFunction205629e9" using statement "UPDATE FUNCS SET FUNC_NAME=? WHERE FUNC_ID=?" failed : org.apache.derby.shared.common.error.DerbySQLIntegrityConstraintViolationException: The statement was aborted because it would have caused a duplicate key value in a unique or primary key constraint or unique index identified by 'UNIQUEFUNCTION' defined on 'FUNCS'.
	at org.apache.derby.impl.jdbc.SQLExceptionFactory.getSQLException(Unknown Source)
	at org.apache.derby.impl.jdbc.Util.generateCsSQLException(Unknown Source)
	at org.apache.derby.impl.jdbc.TransactionResourceImpl.wrapInSQLException(Unknown Source)
	at org.apache.derby.impl.jdbc.TransactionResourceImpl.handleException(Unknown Source)
```

### How was this patch tested?
Improved the existing test cases to check whether the messages are right.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14521 from gatorsmile/functionChecking.
2016-09-02 22:31:01 +08:00
Qifan Pu 03d77af9ec [SPARK-16525] [SQL] Enable Row Based HashMap in HashAggregateExec
## What changes were proposed in this pull request?

This PR is the second step for the following feature:

For hash aggregation in Spark SQL, we use a fast aggregation hashmap to act as a "cache" in order to boost aggregation performance. Previously, the hashmap is backed by a `ColumnarBatch`. This has performance issues when we have wide schema for the aggregation table (large number of key fields or value fields).
In this JIRA, we support another implementation of fast hashmap, which is backed by a `RowBatch`. We then automatically pick between the two implementations based on certain knobs.

In this second-step PR, we enable `RowBasedHashMapGenerator` in `HashAggregateExec`.

## How was this patch tested?

Added tests: `RowBasedAggregateHashMapSuite` and ` VectorizedAggregateHashMapSuite`
Additional micro-benchmarks tests and TPCDS results will be added in a separate PR in the series.

Author: Qifan Pu <qifan.pu@gmail.com>
Author: ooq <qifan.pu@gmail.com>

Closes #14176 from ooq/rowbasedfastaggmap-pr2.
2016-09-01 16:56:35 -07:00
Yucai Yu e388bd5449 [SPARK-16732][SQL] Remove unused codes in subexpressionEliminationForWholeStageCodegen
## What changes were proposed in this pull request?
Some codes in subexpressionEliminationForWholeStageCodegen are never used actually.
Remove them using this PR.

## How was this patch tested?
Local unit tests.

Author: Yucai Yu <yucai.yu@intel.com>

Closes #14366 from yucai/subExpr_unused_codes.
2016-09-01 14:13:38 -07:00
Sean Owen 3893e8c576 [SPARK-17331][CORE][MLLIB] Avoid allocating 0-length arrays
## What changes were proposed in this pull request?

Avoid allocating some 0-length arrays, esp. in UTF8String, and by using Array.empty in Scala over Array[T]()

## How was this patch tested?

Jenkins

Author: Sean Owen <sowen@cloudera.com>

Closes #14895 from srowen/SPARK-17331.
2016-09-01 12:13:07 -07:00
Herman van Hovell 2be5f8d7e0 [SPARK-17263][SQL] Add hexadecimal literal parsing
## What changes were proposed in this pull request?
This PR adds the ability to parse SQL (hexadecimal) binary literals (AKA bit strings). It follows the following syntax `X'[Hexadecimal Characters]+'`, for example: `X'01AB'` would create a binary the following binary array `0x01AB`.

If an uneven number of hexadecimal characters is passed, then the upper 4 bits of the initial byte are kept empty, and the lower 4 bits are filled using the first character. For example `X'1C7'` would create the following binary array `0x01C7`.

Binary data (Array[Byte]) does not have a proper `hashCode` and `equals` functions. This meant that comparing `Literal`s containing binary data was a pain. I have updated Literal.hashCode and Literal.equals to deal properly with binary data.

## How was this patch tested?
Added tests to the `ExpressionParserSuite`, `SQLQueryTestSuite` and `ExpressionSQLBuilderSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #14832 from hvanhovell/SPARK-17263.
2016-09-01 12:01:22 -07:00
Tejas Patil adaaffa34e [SPARK-17271][SQL] Remove redundant semanticEquals() from SortOrder
## What changes were proposed in this pull request?

Removing `semanticEquals()` from `SortOrder` because it can use the `semanticEquals()` provided by its parent class (`Expression`). This was as per suggestion by cloud-fan at 7192418b3a (r77106801)

## How was this patch tested?

Ran the test added in https://github.com/apache/spark/pull/14841

Author: Tejas Patil <tejasp@fb.com>

Closes #14910 from tejasapatil/SPARK-17271_remove_semantic_ordering.
2016-09-01 16:47:37 +08:00
Sean Zhong a18c169fd0 [SPARK-16283][SQL] Implements percentile_approx aggregation function which supports partial aggregation.
## What changes were proposed in this pull request?

This PR implements aggregation function `percentile_approx`. Function `percentile_approx` returns the approximate percentile(s) of a column at the given percentage(s). A percentile is a watermark value below which a given percentage of the column values fall. For example, the percentile of column `col` at percentage 50% is the median value of column `col`.

### Syntax:
```
# Returns percentile at a given percentage value. The approximation error can be reduced by increasing parameter accuracy, at the cost of memory.
percentile_approx(col, percentage [, accuracy])

# Returns percentile value array at given percentage value array
percentile_approx(col, array(percentage1 [, percentage2]...) [, accuracy])
```

### Features:
1. This function supports partial aggregation.
2. The memory consumption is bounded. The larger `accuracy` parameter we choose, we smaller error we get. The default accuracy value is 10000, to match with Hive default setting. Choose a smaller value for smaller memory footprint.
3.  This function supports window function aggregation.

### Example usages:
```
## Returns the 25th percentile value, with default accuracy
SELECT percentile_approx(col, 0.25) FROM table

## Returns an array of percentile value (25th, 50th, 75th), with default accuracy
SELECT percentile_approx(col, array(0.25, 0.5, 0.75)) FROM table

## Returns 25th percentile value, with custom accuracy value 100, larger accuracy parameter yields smaller approximation error
SELECT percentile_approx(col, 0.25, 100) FROM table

## Returns the 25th, and 50th percentile values, with custom accuracy value 100
SELECT percentile_approx(col, array(0.25, 0.5), 100) FROM table
```

### NOTE:
1. The `percentile_approx` implementation is different from Hive, so the result returned on same query maybe slightly different with Hive. This implementation uses `QuantileSummaries` as the underlying probabilistic data structure, and mainly follows paper `Space-efficient Online Computation of Quantile Summaries` by Greenwald, Michael and Khanna, Sanjeev. (http://dx.doi.org/10.1145/375663.375670)`
2. The current implementation of `QuantileSummaries` doesn't support automatic compression. This PR has a rule to do compression automatically at the caller side, but it may not be optimal.

## How was this patch tested?

Unit test, and Sql query test.

## Acknowledgement
1. This PR's work in based on lw-lin's PR https://github.com/apache/spark/pull/14298, with improvements like supporting partial aggregation, fixing out of memory issue.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #14868 from clockfly/appro_percentile_try_2.
2016-09-01 16:31:13 +08:00
Kazuaki Ishizaki d92cd227cf [SPARK-15985][SQL] Eliminate redundant cast from an array without null or a map without null
## What changes were proposed in this pull request?

This PR eliminates redundant cast from an `ArrayType` with `containsNull = false` or a `MapType` with `containsNull = false`.

For example, in `ArrayType` case, current implementation leaves a cast `cast(value#63 as array<double>).toDoubleArray`. However, we can eliminate `cast(value#63 as array<double>)` if we know `value#63` does not include `null`. This PR apply this elimination for `ArrayType` and `MapType` in `SimplifyCasts` at a plan optimization phase.

In summary, we got 1.2-1.3x performance improvements over the code before applying this PR.
Here are performance results of benchmark programs:
```
  test("Read array in Dataset") {
    import sparkSession.implicits._

    val iters = 5
    val n = 1024 * 1024
    val rows = 15

    val benchmark = new Benchmark("Read primnitive array", n)

    val rand = new Random(511)
    val intDS = sparkSession.sparkContext.parallelize(0 until rows, 1)
      .map(i => Array.tabulate(n)(i => i)).toDS()
    intDS.count() // force to create ds
    val lastElement = n - 1
    val randElement = rand.nextInt(lastElement)

    benchmark.addCase(s"Read int array in Dataset", numIters = iters)(iter => {
      val idx0 = randElement
      val idx1 = lastElement
      intDS.map(a => a(0) + a(idx0) + a(idx1)).collect
    })

    val doubleDS = sparkSession.sparkContext.parallelize(0 until rows, 1)
      .map(i => Array.tabulate(n)(i => i.toDouble)).toDS()
    doubleDS.count() // force to create ds

    benchmark.addCase(s"Read double array in Dataset", numIters = iters)(iter => {
      val idx0 = randElement
      val idx1 = lastElement
      doubleDS.map(a => a(0) + a(idx0) + a(idx1)).collect
    })

    benchmark.run()
  }

Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.4
Intel(R) Core(TM) i5-5257U CPU  2.70GHz

without this PR
Read primnitive array:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Read int array in Dataset                      525 /  690          2.0         500.9       1.0X
Read double array in Dataset                   947 / 1209          1.1         902.7       0.6X

with this PR
Read primnitive array:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Read int array in Dataset                      400 /  492          2.6         381.5       1.0X
Read double array in Dataset                   788 /  870          1.3         751.4       0.5X
```

An example program that originally caused this performance issue.
```
val ds = Seq(Array(1.0, 2.0, 3.0), Array(4.0, 5.0, 6.0)).toDS()
val ds2 = ds.map(p => {
     var s = 0.0
     for (i <- 0 to 2) { s += p(i) }
     s
   })
ds2.show
ds2.explain(true)
```

Plans before this PR
```
== Parsed Logical Plan ==
'SerializeFromObject [input[0, double, true] AS value#68]
+- 'MapElements <function1>, obj#67: double
   +- 'DeserializeToObject unresolveddeserializer(upcast(getcolumnbyordinal(0, ArrayType(DoubleType,false)), ArrayType(DoubleType,false), - root class: "scala.Array").toDoubleArray), obj#66: [D
      +- LocalRelation [value#63]

== Analyzed Logical Plan ==
value: double
SerializeFromObject [input[0, double, true] AS value#68]
+- MapElements <function1>, obj#67: double
   +- DeserializeToObject cast(value#63 as array<double>).toDoubleArray, obj#66: [D
      +- LocalRelation [value#63]

== Optimized Logical Plan ==
SerializeFromObject [input[0, double, true] AS value#68]
+- MapElements <function1>, obj#67: double
   +- DeserializeToObject cast(value#63 as array<double>).toDoubleArray, obj#66: [D
      +- LocalRelation [value#63]

== Physical Plan ==
*SerializeFromObject [input[0, double, true] AS value#68]
+- *MapElements <function1>, obj#67: double
   +- *DeserializeToObject cast(value#63 as array<double>).toDoubleArray, obj#66: [D
      +- LocalTableScan [value#63]
```

Plans after this PR
```
== Parsed Logical Plan ==
'SerializeFromObject [input[0, double, true] AS value#6]
+- 'MapElements <function1>, obj#5: double
   +- 'DeserializeToObject unresolveddeserializer(upcast(getcolumnbyordinal(0, ArrayType(DoubleType,false)), ArrayType(DoubleType,false), - root class: "scala.Array").toDoubleArray), obj#4: [D
      +- LocalRelation [value#1]

== Analyzed Logical Plan ==
value: double
SerializeFromObject [input[0, double, true] AS value#6]
+- MapElements <function1>, obj#5: double
   +- DeserializeToObject cast(value#1 as array<double>).toDoubleArray, obj#4: [D
      +- LocalRelation [value#1]

== Optimized Logical Plan ==
SerializeFromObject [input[0, double, true] AS value#6]
+- MapElements <function1>, obj#5: double
   +- DeserializeToObject value#1.toDoubleArray, obj#4: [D
      +- LocalRelation [value#1]

== Physical Plan ==
*SerializeFromObject [input[0, double, true] AS value#6]
+- *MapElements <function1>, obj#5: double
   +- *DeserializeToObject value#1.toDoubleArray, obj#4: [D
      +- LocalTableScan [value#1]
```

## How was this patch tested?

Tested by new test cases in `SimplifyCastsSuite`

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #13704 from kiszk/SPARK-15985.
2016-08-31 12:40:53 +08:00
gatorsmile bca79c8230 [SPARK-17234][SQL] Table Existence Checking when Index Table with the Same Name Exists
### What changes were proposed in this pull request?
Hive Index tables are not supported by Spark SQL. Thus, we issue an exception when users try to access Hive Index tables. When the internal function `tableExists` tries to access Hive Index tables, it always gets the same error message: ```Hive index table is not supported```. This message could be confusing to users, since their SQL operations could be completely unrelated to Hive Index tables. For example, when users try to alter a table to a new name and there exists an index table with the same name, the expected exception should be a `TableAlreadyExistsException`.

This PR made the following changes:
- Introduced a new `AnalysisException` type: `SQLFeatureNotSupportedException`. When users try to access an `Index Table`, we will issue a `SQLFeatureNotSupportedException`.
- `tableExists` returns `true` when hitting a `SQLFeatureNotSupportedException` and the feature is `Hive index table`.
- Add a checking `requireTableNotExists` for `SessionCatalog`'s `createTable` API; otherwise, the current implementation relies on the Hive's internal checking.

### How was this patch tested?
Added a test case

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14801 from gatorsmile/tableExists.
2016-08-30 17:27:00 +08:00
Josh Rosen 48b459ddd5 [SPARK-17301][SQL] Remove unused classTag field from AtomicType base class
There's an unused `classTag` val in the AtomicType base class which is causing unnecessary slowness in deserialization because it needs to grab ScalaReflectionLock and create a new runtime reflection mirror. Removing this unused code gives a small but measurable performance boost in SQL task deserialization.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #14869 from JoshRosen/remove-unused-classtag.
2016-08-30 09:58:00 +08:00
Davies Liu 48caec2516 [SPARK-17063] [SQL] Improve performance of MSCK REPAIR TABLE with Hive metastore
## What changes were proposed in this pull request?

This PR split the the single `createPartitions()` call into smaller batches, which could prevent Hive metastore from OOM (caused by millions of partitions).

It will also try to gather all the fast stats (number of files and total size of all files) in parallel to avoid the bottle neck of listing the files in metastore sequential, which is controlled by spark.sql.gatherFastStats (enabled by default).

## How was this patch tested?

Tested locally with 10000 partitions and 100 files with embedded metastore, without gathering fast stats in parallel, adding partitions took 153 seconds, after enable that, gathering the fast stats took about 34 seconds, adding these partitions took 25 seconds (most of the time spent in object store), 59 seconds in total, 2.5X faster (with larger cluster, gathering will much faster).

Author: Davies Liu <davies@databricks.com>

Closes #14607 from davies/repair_batch.
2016-08-29 11:23:53 -07:00
Tejas Patil 095862a3cf [SPARK-17271][SQL] Planner adds un-necessary Sort even if child ordering is semantically same as required ordering
## What changes were proposed in this pull request?

Jira : https://issues.apache.org/jira/browse/SPARK-17271

Planner is adding un-needed SORT operation due to bug in the way comparison for `SortOrder` is done at https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala#L253
`SortOrder` needs to be compared semantically because `Expression` within two `SortOrder` can be "semantically equal" but not literally equal objects.

eg. In case of `sql("SELECT * FROM table1 a JOIN table2 b ON a.col1=b.col1")`

Expression in required SortOrder:
```
      AttributeReference(
        name = "col1",
        dataType = LongType,
        nullable = false
      ) (exprId = exprId,
        qualifier = Some("a")
      )
```

Expression in child SortOrder:
```
      AttributeReference(
        name = "col1",
        dataType = LongType,
        nullable = false
      ) (exprId = exprId)
```

Notice that the output column has a qualifier but the child attribute does not but the inherent expression is the same and hence in this case we can say that the child satisfies the required sort order.

This PR includes following changes:
- Added a `semanticEquals` method to `SortOrder` so that it can compare underlying child expressions semantically (and not using default Object.equals)
- Fixed `EnsureRequirements` to use semantic comparison of SortOrder

## How was this patch tested?

- Added a test case to `PlannerSuite`. Ran rest tests in `PlannerSuite`

Author: Tejas Patil <tejasp@fb.com>

Closes #14841 from tejasapatil/SPARK-17271_sort_order_equals_bug.
2016-08-28 19:14:58 +02:00
Reynold Xin 718b6bad2d [SPARK-17274][SQL] Move join optimizer rules into a separate file
## What changes were proposed in this pull request?
As part of breaking Optimizer.scala apart, this patch moves various join rules into a single file.

## How was this patch tested?
This should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #14846 from rxin/SPARK-17274.
2016-08-27 00:36:18 -07:00
Reynold Xin 5aad4509c1 [SPARK-17273][SQL] Move expression optimizer rules into a separate file
## What changes were proposed in this pull request?
As part of breaking Optimizer.scala apart, this patch moves various expression optimization rules into a single file.

## How was this patch tested?
This should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #14845 from rxin/SPARK-17273.
2016-08-27 00:34:35 -07:00
Reynold Xin 0243b32873 [SPARK-17272][SQL] Move subquery optimizer rules into its own file
## What changes were proposed in this pull request?
As part of breaking Optimizer.scala apart, this patch moves various subquery rules into a single file.

## How was this patch tested?
This should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #14844 from rxin/SPARK-17272.
2016-08-27 00:32:57 -07:00
Reynold Xin dcefac4387 [SPARK-17269][SQL] Move finish analysis optimization stage into its own file
## What changes were proposed in this pull request?
As part of breaking Optimizer.scala apart, this patch moves various finish analysis optimization stage rules into a single file. I'm submitting separate pull requests so we can more easily merge this in branch-2.0 to simplify optimizer backports.

## How was this patch tested?
This should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #14838 from rxin/SPARK-17269.
2016-08-26 22:10:28 -07:00
Reynold Xin cc0caa690b [SPARK-17270][SQL] Move object optimization rules into its own file
## What changes were proposed in this pull request?
As part of breaking Optimizer.scala apart, this patch moves various Dataset object optimization rules into a single file. I'm submitting separate pull requests so we can more easily merge this in branch-2.0 to simplify optimizer backports.

## How was this patch tested?
This should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #14839 from rxin/SPARK-17270.
2016-08-26 21:41:58 -07:00
Sameer Agarwal 540e912801 [SPARK-17244] Catalyst should not pushdown non-deterministic join conditions
## What changes were proposed in this pull request?

Given that non-deterministic expressions can be stateful, pushing them down the query plan during the optimization phase can cause incorrect behavior. This patch fixes that issue by explicitly disabling that.

## How was this patch tested?

A new test in `FilterPushdownSuite` that checks catalyst behavior for both deterministic and non-deterministic join conditions.

Author: Sameer Agarwal <sameerag@cs.berkeley.edu>

Closes #14815 from sameeragarwal/constraint-inputfile.
2016-08-26 16:40:59 -07:00
Herman van Hovell a11d10f182 [SPARK-17246][SQL] Add BigDecimal literal
## What changes were proposed in this pull request?
This PR adds parser support for `BigDecimal` literals. If you append the suffix `BD` to a valid number then this will be interpreted as a `BigDecimal`, for example `12.0E10BD` will interpreted into a BigDecimal with scale -9 and precision 3. This is useful in situations where you need exact values.

## How was this patch tested?
Added tests to `ExpressionParserSuite`, `ExpressionSQLBuilderSuite` and `SQLQueryTestSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #14819 from hvanhovell/SPARK-17246.
2016-08-26 13:29:22 -07:00
Wenchen Fan 970ab8f6dd [SPARK-17187][SQL][FOLLOW-UP] improve document of TypedImperativeAggregate
## What changes were proposed in this pull request?

improve the document to make it easier to understand and also mention window operator.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14822 from cloud-fan/object-agg.
2016-08-26 10:56:57 -07:00
hyukjinkwon b964a172a8 [SPARK-17212][SQL] TypeCoercion supports widening conversion between DateType and TimestampType
## What changes were proposed in this pull request?

Currently, type-widening does not work between `TimestampType` and `DateType`.

This applies to `SetOperation`, `Union`, `In`, `CaseWhen`, `Greatest`,  `Leatest`, `CreateArray`, `CreateMap`, `Coalesce`, `NullIf`, `IfNull`, `Nvl` and `Nvl2`, .

This PR adds the support for widening `DateType` to `TimestampType` for them.

For a simple example,

**Before**

```scala
Seq(Tuple2(new Timestamp(0), new Date(0))).toDF("a", "b").selectExpr("greatest(a, b)").show()
```

shows below:

```
cannot resolve 'greatest(`a`, `b`)' due to data type mismatch: The expressions should all have the same type, got GREATEST(timestamp, date)
```

or union as below:

```scala
val a = Seq(Tuple1(new Timestamp(0))).toDF()
val b = Seq(Tuple1(new Date(0))).toDF()
a.union(b).show()
```

shows below:

```
Union can only be performed on tables with the compatible column types. DateType <> TimestampType at the first column of the second table;
```

**After**

```scala
Seq(Tuple2(new Timestamp(0), new Date(0))).toDF("a", "b").selectExpr("greatest(a, b)").show()
```

shows below:

```
+----------------------------------------------------+
|greatest(CAST(a AS TIMESTAMP), CAST(b AS TIMESTAMP))|
+----------------------------------------------------+
|                                1969-12-31 16:00:...|
+----------------------------------------------------+
```

or union as below:

```scala
val a = Seq(Tuple1(new Timestamp(0))).toDF()
val b = Seq(Tuple1(new Date(0))).toDF()
a.union(b).show()
```

shows below:

```
+--------------------+
|                  _1|
+--------------------+
|1969-12-31 16:00:...|
|1969-12-31 00:00:...|
+--------------------+
```

## How was this patch tested?

Unit tests in `TypeCoercionSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>
Author: HyukjinKwon <gurwls223@gmail.com>

Closes #14786 from HyukjinKwon/SPARK-17212.
2016-08-26 08:58:43 +08:00
Sean Zhong d96d151563 [SPARK-17187][SQL] Supports using arbitrary Java object as internal aggregation buffer object
## What changes were proposed in this pull request?

This PR introduces an abstract class `TypedImperativeAggregate` so that an aggregation function of TypedImperativeAggregate can use  **arbitrary** user-defined Java object as intermediate aggregation buffer object.

**This has advantages like:**
1. It now can support larger category of aggregation functions. For example, it will be much easier to implement aggregation function `percentile_approx`, which has a complex aggregation buffer definition.
2. It can be used to avoid doing serialization/de-serialization for every call of `update` or `merge` when converting domain specific aggregation object to internal Spark-Sql storage format.
3. It is easier to integrate with other existing monoid libraries like algebird, and supports more aggregation functions with high performance.

Please see `org.apache.spark.sql.TypedImperativeAggregateSuite.TypedMaxAggregate` to find an example of how to defined a `TypedImperativeAggregate` aggregation function.
Please see Java doc of `TypedImperativeAggregate` and Jira ticket SPARK-17187 for more information.

## How was this patch tested?

Unit tests.

Author: Sean Zhong <seanzhong@databricks.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #14753 from clockfly/object_aggregation_buffer_try_2.
2016-08-25 16:36:16 -07:00
Josh Rosen 3e4c7db4d1 [SPARK-17205] Literal.sql should handle Infinity and NaN
This patch updates `Literal.sql` to properly generate SQL for `NaN` and `Infinity` float and double literals: these special values need to be handled differently from regular values, since simply appending a suffix to the value's `toString()` representation will not work for these values.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #14777 from JoshRosen/SPARK-17205.
2016-08-26 00:15:01 +02:00
gatorsmile d2ae6399ee [SPARK-16991][SPARK-17099][SPARK-17120][SQL] Fix Outer Join Elimination when Filter's isNotNull Constraints Unable to Filter Out All Null-supplying Rows
### What changes were proposed in this pull request?
This PR is to fix an incorrect outer join elimination when filter's `isNotNull` constraints is unable to filter out all null-supplying rows. For example, `isnotnull(coalesce(b#227, c#238))`.

Users can hit this error when they try to use `using/natural outer join`, which is converted to a normal outer join with a `coalesce` expression on the `using columns`. For example,
```Scala
    val a = Seq((1, 2), (2, 3)).toDF("a", "b")
    val b = Seq((2, 5), (3, 4)).toDF("a", "c")
    val c = Seq((3, 1)).toDF("a", "d")
    val ab = a.join(b, Seq("a"), "fullouter")
    ab.join(c, "a").explain(true)
```
The dataframe `ab` is doing `using full-outer join`, which is converted to a normal outer join with a `coalesce` expression. Constraints inference generates a `Filter` with constraints `isnotnull(coalesce(b#227, c#238))`. Then, it triggers a wrong outer join elimination and generates a wrong result.
```
Project [a#251, b#227, c#237, d#247]
+- Join Inner, (a#251 = a#246)
   :- Project [coalesce(a#226, a#236) AS a#251, b#227, c#237]
   :  +- Join FullOuter, (a#226 = a#236)
   :     :- Project [_1#223 AS a#226, _2#224 AS b#227]
   :     :  +- LocalRelation [_1#223, _2#224]
   :     +- Project [_1#233 AS a#236, _2#234 AS c#237]
   :        +- LocalRelation [_1#233, _2#234]
   +- Project [_1#243 AS a#246, _2#244 AS d#247]
      +- LocalRelation [_1#243, _2#244]

== Optimized Logical Plan ==
Project [a#251, b#227, c#237, d#247]
+- Join Inner, (a#251 = a#246)
   :- Project [coalesce(a#226, a#236) AS a#251, b#227, c#237]
   :  +- Filter isnotnull(coalesce(a#226, a#236))
   :     +- Join FullOuter, (a#226 = a#236)
   :        :- LocalRelation [a#226, b#227]
   :        +- LocalRelation [a#236, c#237]
   +- LocalRelation [a#246, d#247]
```

**A note to the `Committer`**, please also give the credit to dongjoon-hyun who submitted another PR for fixing this issue. https://github.com/apache/spark/pull/14580

### How was this patch tested?
Added test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14661 from gatorsmile/fixOuterJoinElimination.
2016-08-25 14:18:58 +02:00
Liwei Lin e0b20f9f24 [SPARK-17061][SPARK-17093][SQL] MapObjects` should make copies of unsafe-backed data
## What changes were proposed in this pull request?

Currently `MapObjects` does not make copies of unsafe-backed data, leading to problems like [SPARK-17061](https://issues.apache.org/jira/browse/SPARK-17061) [SPARK-17093](https://issues.apache.org/jira/browse/SPARK-17093).

This patch makes `MapObjects` make copies of unsafe-backed data.

Generated code - prior to this patch:
```java
...
/* 295 */ if (isNull12) {
/* 296 */   convertedArray1[loopIndex1] = null;
/* 297 */ } else {
/* 298 */   convertedArray1[loopIndex1] = value12;
/* 299 */ }
...
```

Generated code - after this patch:
```java
...
/* 295 */ if (isNull12) {
/* 296 */   convertedArray1[loopIndex1] = null;
/* 297 */ } else {
/* 298 */   convertedArray1[loopIndex1] = value12 instanceof UnsafeRow? value12.copy() : value12;
/* 299 */ }
...
```

## How was this patch tested?

Add a new test case which would fail without this patch.

Author: Liwei Lin <lwlin7@gmail.com>

Closes #14698 from lw-lin/mapobjects-copy.
2016-08-25 11:24:40 +02:00
gatorsmile 4d0706d616 [SPARK-17190][SQL] Removal of HiveSharedState
### What changes were proposed in this pull request?
Since `HiveClient` is used to interact with the Hive metastore, it should be hidden in `HiveExternalCatalog`. After moving `HiveClient` into `HiveExternalCatalog`, `HiveSharedState` becomes a wrapper of `HiveExternalCatalog`. Thus, removal of `HiveSharedState` becomes straightforward. After removal of `HiveSharedState`, the reflection logic is directly applied on the choice of `ExternalCatalog` types, based on the configuration of `CATALOG_IMPLEMENTATION`.

~~`HiveClient` is also used/invoked by the other entities besides HiveExternalCatalog, we defines the following two APIs: getClient and getNewClient~~

### How was this patch tested?
The existing test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14757 from gatorsmile/removeHiveClient.
2016-08-25 12:50:03 +08:00
Sameer Agarwal ac27557eb6 [SPARK-17228][SQL] Not infer/propagate non-deterministic constraints
## What changes were proposed in this pull request?

Given that filters based on non-deterministic constraints shouldn't be pushed down in the query plan, unnecessarily inferring them is confusing and a source of potential bugs. This patch simplifies the inferring logic by simply ignoring them.

## How was this patch tested?

Added a new test in `ConstraintPropagationSuite`.

Author: Sameer Agarwal <sameerag@cs.berkeley.edu>

Closes #14795 from sameeragarwal/deterministic-constraints.
2016-08-24 21:24:24 -07:00
Dongjoon Hyun 40b30fcf45 [SPARK-16983][SQL] Add prettyName for row_number, dense_rank, percent_rank, cume_dist
## What changes were proposed in this pull request?

Currently, two-word window functions like `row_number`, `dense_rank`, `percent_rank`, and `cume_dist` are expressed without `_` in error messages. We had better show the correct names.

**Before**
```scala
scala> sql("select row_number()").show
java.lang.UnsupportedOperationException: Cannot evaluate expression: rownumber()
```

**After**
```scala
scala> sql("select row_number()").show
java.lang.UnsupportedOperationException: Cannot evaluate expression: row_number()
```

## How was this patch tested?

Pass the Jenkins and manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14571 from dongjoon-hyun/SPARK-16983.
2016-08-24 21:14:40 +02:00
Wenchen Fan 52fa45d62a [SPARK-17186][SQL] remove catalog table type INDEX
## What changes were proposed in this pull request?

Actually Spark SQL doesn't support index, the catalog table type `INDEX` is from Hive. However, most operations in Spark SQL can't handle index table, e.g. create table, alter table, etc.

Logically index table should be invisible to end users, and Hive also generates special table name for index table to avoid users accessing it directly. Hive has special SQL syntax to create/show/drop index tables.

At Spark SQL side, although we can describe index table directly, but the result is unreadable, we should use the dedicated SQL syntax to do it(e.g. `SHOW INDEX ON tbl`). Spark SQL can also read index table directly, but the result is always empty.(Can hive read index table directly?)

This PR remove the table type `INDEX`, to make it clear that Spark SQL doesn't support index currently.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14752 from cloud-fan/minor2.
2016-08-23 23:46:09 -07:00
Josh Rosen bf8ff833e3 [SPARK-17194] Use single quotes when generating SQL for string literals
When Spark emits SQL for a string literal, it should wrap the string in single quotes, not double quotes. Databases which adhere more strictly to the ANSI SQL standards, such as Postgres, allow only single-quotes to be used for denoting string literals (see http://stackoverflow.com/a/1992331/590203).

Author: Josh Rosen <joshrosen@databricks.com>

Closes #14763 from JoshRosen/SPARK-17194.
2016-08-23 22:31:58 +02:00
Jacek Laskowski 9d376ad76c [SPARK-17199] Use CatalystConf.resolver for case-sensitivity comparison
## What changes were proposed in this pull request?

Use `CatalystConf.resolver` consistently for case-sensitivity comparison (removed dups).

## How was this patch tested?

Local build. Waiting for Jenkins to ensure clean build and test.

Author: Jacek Laskowski <jacek@japila.pl>

Closes #14771 from jaceklaskowski/17199-catalystconf-resolver.
2016-08-23 12:59:25 +02:00
Sean Zhong cc33460a51 [SPARK-17188][SQL] Moves class QuantileSummaries to project catalyst for implementing percentile_approx
## What changes were proposed in this pull request?

This is a sub-task of [SPARK-16283](https://issues.apache.org/jira/browse/SPARK-16283) (Implement percentile_approx SQL function), which moves class QuantileSummaries to project catalyst so that it can be reused when implementing aggregation function `percentile_approx`.

## How was this patch tested?

This PR only does class relocation, class implementation is not changed.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #14754 from clockfly/move_QuantileSummaries_to_catalyst.
2016-08-23 14:57:00 +08:00
Cheng Lian 2cdd92a7cd [SPARK-17182][SQL] Mark Collect as non-deterministic
## What changes were proposed in this pull request?

This PR marks the abstract class `Collect` as non-deterministic since the results of `CollectList` and `CollectSet` depend on the actual order of input rows.

## How was this patch tested?

Existing test cases should be enough.

Author: Cheng Lian <lian@databricks.com>

Closes #14749 from liancheng/spark-17182-non-deterministic-collect.
2016-08-23 09:11:47 +08:00
Eric Liang 84770b59f7 [SPARK-17162] Range does not support SQL generation
## What changes were proposed in this pull request?

The range operator previously didn't support SQL generation, which made it not possible to use in views.

## How was this patch tested?

Unit tests.

cc hvanhovell

Author: Eric Liang <ekl@databricks.com>

Closes #14724 from ericl/spark-17162.
2016-08-22 15:48:35 -07:00
Davies Liu 8d35a6f68d [SPARK-17115][SQL] decrease the threshold when split expressions
## What changes were proposed in this pull request?

In 2.0, we change the threshold of splitting expressions from 16K to 64K, which cause very bad performance on wide table, because the generated method can't be JIT compiled by default (above the limit of 8K bytecode).

This PR will decrease it to 1K, based on the benchmark results for a wide table with 400 columns of LongType.

It also fix a bug around splitting expression in whole-stage codegen (it should not split them).

## How was this patch tested?

Added benchmark suite.

Author: Davies Liu <davies@databricks.com>

Closes #14692 from davies/split_exprs.
2016-08-22 16:16:03 +08:00
Dongjoon Hyun 91c2397684 [SPARK-17098][SQL] Fix NullPropagation optimizer to handle COUNT(NULL) OVER correctly
## What changes were proposed in this pull request?

Currently, `NullPropagation` optimizer replaces `COUNT` on null literals in a bottom-up fashion. During that, `WindowExpression` is not covered properly. This PR adds the missing propagation logic.

**Before**
```scala
scala> sql("SELECT COUNT(1 + NULL) OVER ()").show
java.lang.UnsupportedOperationException: Cannot evaluate expression: cast(0 as bigint) windowspecdefinition(ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)
```

**After**
```scala
scala> sql("SELECT COUNT(1 + NULL) OVER ()").show
+----------------------------------------------------------------------------------------------+
|count((1 + CAST(NULL AS INT))) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)|
+----------------------------------------------------------------------------------------------+
|                                                                                             0|
+----------------------------------------------------------------------------------------------+
```

## How was this patch tested?

Pass the Jenkins test with a new test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14689 from dongjoon-hyun/SPARK-17098.
2016-08-21 22:07:47 +02:00
petermaxlee 45d40d9f66 [SPARK-17150][SQL] Support SQL generation for inline tables
## What changes were proposed in this pull request?
This patch adds support for SQL generation for inline tables. With this, it would be possible to create a view that depends on inline tables.

## How was this patch tested?
Added a test case in LogicalPlanToSQLSuite.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14709 from petermaxlee/SPARK-17150.
2016-08-20 13:19:38 +08:00
Srinath Shankar ba1737c21a [SPARK-17158][SQL] Change error message for out of range numeric literals
## What changes were proposed in this pull request?

Modifies error message for numeric literals to
Numeric literal <literal> does not fit in range [min, max] for type <T>

## How was this patch tested?

Fixed up the error messages for literals.sql in  SqlQueryTestSuite and re-ran via sbt. Also fixed up error messages in ExpressionParserSuite

Author: Srinath Shankar <srinath@databricks.com>

Closes #14721 from srinathshankar/sc4296.
2016-08-19 19:54:26 -07:00
petermaxlee a117afa7c2 [SPARK-17149][SQL] array.sql for testing array related functions
## What changes were proposed in this pull request?
This patch creates array.sql in SQLQueryTestSuite for testing array related functions, including:

- indexing
- array creation
- size
- array_contains
- sort_array

## How was this patch tested?
The patch itself is about adding tests.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14708 from petermaxlee/SPARK-17149.
2016-08-19 18:14:45 -07:00
Reynold Xin 67e59d464f [SPARK-16994][SQL] Whitelist operators for predicate pushdown
## What changes were proposed in this pull request?
This patch changes predicate pushdown optimization rule (PushDownPredicate) from using a blacklist to a whitelist. That is to say, operators must be explicitly allowed. This approach is more future-proof: previously it was possible for us to introduce a new operator and then render the optimization rule incorrect.

This also fixes the bug that previously we allowed pushing filter beneath limit, which was incorrect. That is to say, before this patch, the optimizer would rewrite
```
select * from (select * from range(10) limit 5) where id > 3

to

select * from range(10) where id > 3 limit 5
```

## How was this patch tested?
- a unit test case in FilterPushdownSuite
- an end-to-end test in limit.sql

Author: Reynold Xin <rxin@databricks.com>

Closes #14713 from rxin/SPARK-16994.
2016-08-19 21:11:35 +08:00
Reynold Xin b482c09fa2 HOTFIX: compilation broken due to protected ctor. 2016-08-18 19:02:32 -07:00
petermaxlee f5472dda51 [SPARK-16947][SQL] Support type coercion and foldable expression for inline tables
## What changes were proposed in this pull request?
This patch improves inline table support with the following:

1. Support type coercion.
2. Support using foldable expressions. Previously only literals were supported.
3. Improve error message handling.
4. Improve test coverage.

## How was this patch tested?
Added a new unit test suite ResolveInlineTablesSuite and a new file-based end-to-end test inline-table.sql.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14676 from petermaxlee/SPARK-16947.
2016-08-19 09:19:47 +08:00
petermaxlee 68f5087d21 [SPARK-17117][SQL] 1 / NULL should not fail analysis
## What changes were proposed in this pull request?
This patch fixes the problem described in SPARK-17117, i.e. "SELECT 1 / NULL" throws an analysis exception:

```
org.apache.spark.sql.AnalysisException: cannot resolve '(1 / NULL)' due to data type mismatch: differing types in '(1 / NULL)' (int and null).
```

The problem is that division type coercion did not take null type into account.

## How was this patch tested?
A unit test for the type coercion, and a few end-to-end test cases using SQLQueryTestSuite.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14695 from petermaxlee/SPARK-17117.
2016-08-18 13:44:13 +02:00
Eric Liang 412dba63b5 [SPARK-17069] Expose spark.range() as table-valued function in SQL
## What changes were proposed in this pull request?

This adds analyzer rules for resolving table-valued functions, and adds one builtin implementation for range(). The arguments for range() are the same as those of `spark.range()`.

## How was this patch tested?

Unit tests.

cc hvanhovell

Author: Eric Liang <ekl@databricks.com>

Closes #14656 from ericl/sc-4309.
2016-08-18 13:33:55 +02:00
Liang-Chi Hsieh e82dbe600e [SPARK-17107][SQL] Remove redundant pushdown rule for Union
## What changes were proposed in this pull request?

The `Optimizer` rules `PushThroughSetOperations` and `PushDownPredicate` have a redundant rule to push down `Filter` through `Union`. We should remove it.

## How was this patch tested?

Jenkins tests.

Author: Liang-Chi Hsieh <simonh@tw.ibm.com>

Closes #14687 from viirya/remove-extra-pushdown.
2016-08-18 12:45:56 +02:00
petermaxlee 3e6ef2e8a4 [SPARK-17034][SQL] Minor code cleanup for UnresolvedOrdinal
## What changes were proposed in this pull request?
I was looking at the code for UnresolvedOrdinal and made a few small changes to make it slightly more clear:

1. Rename the rule to SubstituteUnresolvedOrdinals which is more consistent with other rules that start with verbs. Note that this is still inconsistent with CTESubstitution and WindowsSubstitution.
2. Broke the test suite down from a single test case to three test cases.

## How was this patch tested?
This is a minor cleanup.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14672 from petermaxlee/SPARK-17034.
2016-08-18 16:17:01 +08:00
Liang-Chi Hsieh 10204b9d29 [SPARK-16995][SQL] TreeNodeException when flat mapping RelationalGroupedDataset created from DataFrame containing a column created with lit/expr
## What changes were proposed in this pull request?

A TreeNodeException is thrown when executing the following minimal example in Spark 2.0.

    import spark.implicits._
    case class test (x: Int, q: Int)

    val d = Seq(1).toDF("x")
    d.withColumn("q", lit(0)).as[test].groupByKey(_.x).flatMapGroups{case (x, iter) => List[Int]()}.show
    d.withColumn("q", expr("0")).as[test].groupByKey(_.x).flatMapGroups{case (x, iter) => List[Int]()}.show

The problem is at `FoldablePropagation`. The rule will do `transformExpressions` on `LogicalPlan`. The query above contains a `MapGroups` which has a parameter `dataAttributes:Seq[Attribute]`. One attributes in `dataAttributes` will be transformed to an `Alias(literal(0), _)` in `FoldablePropagation`. `Alias` is not an `Attribute` and causes the error.

We can't easily detect such type inconsistency during transforming expressions. A direct approach to this problem is to skip doing `FoldablePropagation` on object operators as they should not contain such expressions.

## How was this patch tested?

Jenkins tests.

Author: Liang-Chi Hsieh <simonh@tw.ibm.com>

Closes #14648 from viirya/flat-mapping.
2016-08-18 13:24:12 +08:00
Herman van Hovell 0b0c8b95e3 [SPARK-17106] [SQL] Simplify the SubqueryExpression interface
## What changes were proposed in this pull request?
The current subquery expression interface contains a little bit of technical debt in the form of a few different access paths to get and set the query contained by the expression. This is confusing to anyone who goes over this code.

This PR unifies these access paths.

## How was this patch tested?
(Existing tests)

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #14685 from hvanhovell/SPARK-17106.
2016-08-17 07:03:24 -07:00
Kazuaki Ishizaki 56d86742d2 [SPARK-15285][SQL] Generated SpecificSafeProjection.apply method grows beyond 64 KB
## What changes were proposed in this pull request?

This PR splits the generated code for ```SafeProjection.apply``` by using ```ctx.splitExpressions()```. This is because the large code body for ```NewInstance``` may grow beyond 64KB bytecode size for ```apply()``` method.

Here is [the original PR](https://github.com/apache/spark/pull/13243) for SPARK-15285. However, it breaks a build with Scala 2.10 since Scala 2.10 does not a case class with large number of members. Thus, it was reverted by [this commit](fa244e5a90).

## How was this patch tested?

Added new tests by using `DefinedByConstructorParams` instead of case class for scala-2.10

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #14670 from kiszk/SPARK-15285-2.
2016-08-17 21:34:57 +08:00
jiangxingbo 4d0cc84afc [SPARK-17032][SQL] Add test cases for methods in ParserUtils.
## What changes were proposed in this pull request?

Currently methods in `ParserUtils` are tested indirectly, we should add test cases in `ParserUtilsSuite` to verify their integrity directly.

## How was this patch tested?

New test cases in `ParserUtilsSuite`

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #14620 from jiangxb1987/parserUtils.
2016-08-17 14:22:36 +02:00
Herman van Hovell f7c9ff57c1 [SPARK-17068][SQL] Make view-usage visible during analysis
## What changes were proposed in this pull request?
This PR adds a field to subquery alias in order to make the usage of views in a resolved `LogicalPlan` more visible (and more understandable).

For example, the following view and query:
```sql
create view constants as select 1 as id union all select 1 union all select 42
select * from constants;
```
...now yields the following analyzed plan:
```
Project [id#39]
+- SubqueryAlias c, `default`.`constants`
   +- Project [gen_attr_0#36 AS id#39]
      +- SubqueryAlias gen_subquery_0
         +- Union
            :- Union
            :  :- Project [1 AS gen_attr_0#36]
            :  :  +- OneRowRelation$
            :  +- Project [1 AS gen_attr_1#37]
            :     +- OneRowRelation$
            +- Project [42 AS gen_attr_2#38]
               +- OneRowRelation$
```
## How was this patch tested?
Added tests for the two code paths in `SessionCatalogSuite` (sql/core) and `HiveMetastoreCatalogSuite` (sql/hive)

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #14657 from hvanhovell/SPARK-17068.
2016-08-16 23:09:53 -07:00
Herman van Hovell 4a2c375be2 [SPARK-17084][SQL] Rename ParserUtils.assert to validate
## What changes were proposed in this pull request?
This PR renames `ParserUtils.assert` to `ParserUtils.validate`. This is done because this method is used to check requirements, and not to check if the program is in an invalid state.

## How was this patch tested?
Simple rename. Compilation should do.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #14665 from hvanhovell/SPARK-17084.
2016-08-16 21:35:39 -07:00
Sean Zhong 7b65030e7a [SPARK-17034][SQL] adds expression UnresolvedOrdinal to represent the ordinals in GROUP BY or ORDER BY
## What changes were proposed in this pull request?

This PR adds expression `UnresolvedOrdinal` to represent the ordinal in GROUP BY or ORDER BY, and fixes the rules when resolving ordinals.

Ordinals in GROUP BY or ORDER BY like `1` in `order by 1` or `group by 1` should be considered as unresolved before analysis. But in current code, it uses `Literal` expression to store the ordinal. This is inappropriate as `Literal` itself is a resolved expression, it gives the user a wrong message that the ordinals has already been resolved.

### Before this change

Ordinal is stored as `Literal` expression

```
scala> sc.setLogLevel("TRACE")
scala> sql("select a from t group by 1 order by 1")
...
'Sort [1 ASC], true
 +- 'Aggregate [1], ['a]
     +- 'UnresolvedRelation `t
```

For query:

```
scala> Seq(1).toDF("a").createOrReplaceTempView("t")
scala> sql("select count(a), a from t group by 2 having a > 0").show
```

During analysis, the intermediate plan before applying rule `ResolveAggregateFunctions` is:

```
'Filter ('a > 0)
   +- Aggregate [2], [count(1) AS count(1)#83L, a#81]
        +- LocalRelation [value#7 AS a#9]
```

Before this PR, rule `ResolveAggregateFunctions` believes all expressions of `Aggregate` have already been resolved, and tries to resolve the expressions in `Filter` directly. But this is wrong, as ordinal `2` in Aggregate is not really resolved!

### After this change

Ordinals are stored as `UnresolvedOrdinal`.

```
scala> sc.setLogLevel("TRACE")
scala> sql("select a from t group by 1 order by 1")
...
'Sort [unresolvedordinal(1) ASC], true
 +- 'Aggregate [unresolvedordinal(1)], ['a]
      +- 'UnresolvedRelation `t`
```

## How was this patch tested?

Unit tests.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #14616 from clockfly/spark-16955.
2016-08-16 15:51:30 +08:00
Dongjoon Hyun 2a105134e9 [SPARK-16771][SQL] WITH clause should not fall into infinite loop.
## What changes were proposed in this pull request?

This PR changes the CTE resolving rule to use only **forward-declared** tables in order to prevent infinite loops. More specifically, new logic is like the following.

* Resolve CTEs in `WITH` clauses first before replacing the main SQL body.
* When resolving CTEs, only forward-declared CTEs or base tables are referenced.
  - Self-referencing is not allowed any more.
  - Cross-referencing is not allowed any more.

**Reported Error Scenarios**
```scala
scala> sql("WITH t AS (SELECT 1 FROM t) SELECT * FROM t")
java.lang.StackOverflowError
...
scala> sql("WITH t1 AS (SELECT * FROM t2), t2 AS (SELECT 2 FROM t1) SELECT * FROM t1, t2")
java.lang.StackOverflowError
...
```
Note that `t`, `t1`, and `t2` are not declared in database. Spark falls into infinite loops before resolving table names.

## How was this patch tested?

Pass the Jenkins tests with new two testcases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14397 from dongjoon-hyun/SPARK-16771-TREENODE.
2016-08-12 19:07:34 +02:00
gatorsmile 79e2caa132 [SPARK-16598][SQL][TEST] Added a test case for verifying the table identifier parsing
#### What changes were proposed in this pull request?
So far, the test cases of `TableIdentifierParserSuite` do not cover the quoted cases. We should add one for avoiding regression.

#### How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14244 from gatorsmile/quotedIdentifiers.
2016-08-12 10:02:00 +01:00
petermaxlee 00e103a6ed [SPARK-17013][SQL] Parse negative numeric literals
## What changes were proposed in this pull request?
This patch updates the SQL parser to parse negative numeric literals as numeric literals, instead of unary minus of positive literals.

This allows the parser to parse the minimal value for each data type, e.g. "-32768S".

## How was this patch tested?
Updated test cases.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14608 from petermaxlee/SPARK-17013.
2016-08-11 23:56:55 -07:00
Davies Liu 0f72e4f04b [SPARK-16958] [SQL] Reuse subqueries within the same query
## What changes were proposed in this pull request?

There could be multiple subqueries that generate same results, we could re-use the result instead of running it multiple times.

This PR also cleanup up how we run subqueries.

For SQL query
```sql
select id,(select avg(id) from t) from t where id > (select avg(id) from t)
```
The explain is
```
== Physical Plan ==
*Project [id#15L, Subquery subquery29 AS scalarsubquery()#35]
:  +- Subquery subquery29
:     +- *HashAggregate(keys=[], functions=[avg(id#15L)])
:        +- Exchange SinglePartition
:           +- *HashAggregate(keys=[], functions=[partial_avg(id#15L)])
:              +- *Range (0, 1000, splits=4)
+- *Filter (cast(id#15L as double) > Subquery subquery29)
   :  +- Subquery subquery29
   :     +- *HashAggregate(keys=[], functions=[avg(id#15L)])
   :        +- Exchange SinglePartition
   :           +- *HashAggregate(keys=[], functions=[partial_avg(id#15L)])
   :              +- *Range (0, 1000, splits=4)
   +- *Range (0, 1000, splits=4)
```
The visualized plan:

![reuse-subquery](https://cloud.githubusercontent.com/assets/40902/17573229/e578d93c-5f0d-11e6-8a3c-0150d81d3aed.png)

## How was this patch tested?

Existing tests.

Author: Davies Liu <davies@databricks.com>

Closes #14548 from davies/subq.
2016-08-11 09:47:19 -07:00
petermaxlee a7b02db457 [SPARK-17015][SQL] group-by/order-by ordinal and arithmetic tests
## What changes were proposed in this pull request?
This patch adds three test files:
1. arithmetic.sql.out
2. order-by-ordinal.sql
3. group-by-ordinal.sql

This includes https://github.com/apache/spark/pull/14594.

## How was this patch tested?
This is a test case change.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14595 from petermaxlee/SPARK-17015.
2016-08-11 01:43:08 -07:00
Dongjoon Hyun 41a7dbdd34 [SPARK-10601][SQL] Support MINUS set operator
## What changes were proposed in this pull request?

This PR adds `MINUS` set operator which is equivalent `EXCEPT DISTINCT`. This will slightly improve the compatibility with Oracle.

## How was this patch tested?

Pass the Jenkins with newly added testcases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14570 from dongjoon-hyun/SPARK-10601.
2016-08-10 10:31:30 +02:00
Michał Kiełbowicz 9dc3e602d7 Fixed typo
## What changes were proposed in this pull request?

Fixed small typo - "value ... ~~in~~ is null"

## How was this patch tested?

Still compiles!

Author: Michał Kiełbowicz <jupblb@users.noreply.github.com>

Closes #14569 from jupblb/typo-fix.
2016-08-09 23:01:50 -07:00
Davies Liu 92da22878b [SPARK-16905] SQL DDL: MSCK REPAIR TABLE
## What changes were proposed in this pull request?

MSCK REPAIR TABLE could be used to recover the partitions in external catalog based on partitions in file system.

Another syntax is: ALTER TABLE table RECOVER PARTITIONS

The implementation in this PR will only list partitions (not the files with a partition) in driver (in parallel if needed).

## How was this patch tested?

Added unit tests for it and Hive compatibility test suite.

Author: Davies Liu <davies@databricks.com>

Closes #14500 from davies/repair_table.
2016-08-09 10:04:36 -07:00
Sean Zhong bca43cd635 [SPARK-16898][SQL] Adds argument type information for typed logical plan like MapElements, TypedFilter, and AppendColumn
## What changes were proposed in this pull request?

This PR adds argument type information for typed logical plan like MapElements, TypedFilter, and AppendColumn, so that we can use these info in customized optimizer rule.

## How was this patch tested?

Existing test.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #14494 from clockfly/add_more_info_for_typed_operator.
2016-08-09 08:36:50 +08:00
Holden Karau 9216901d52 [SPARK-16779][TRIVIAL] Avoid using postfix operators where they do not add much and remove whitelisting
## What changes were proposed in this pull request?

Avoid using postfix operation for command execution in SQLQuerySuite where it wasn't whitelisted and audit existing whitelistings removing postfix operators from most places. Some notable places where postfix operation remains is in the XML parsing & time units (seconds, millis, etc.) where it arguably can improve readability.

## How was this patch tested?

Existing tests.

Author: Holden Karau <holden@us.ibm.com>

Closes #14407 from holdenk/SPARK-16779.
2016-08-08 15:54:03 -07:00
gatorsmile 5959df217d [SPARK-16936][SQL] Case Sensitivity Support for Refresh Temp Table
### What changes were proposed in this pull request?
Currently, the `refreshTable` API is always case sensitive.

When users use the view name without the exact case match, the API silently ignores the call. Users might expect the command has been successfully completed. However, when users run the subsequent SQL commands, they might still get the exception, like
```
Job aborted due to stage failure:
Task 1 in stage 4.0 failed 1 times, most recent failure: Lost task 1.0 in stage 4.0 (TID 7, localhost):
java.io.FileNotFoundException:
File file:/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-bd4b9ea6-9aec-49c5-8f05-01cff426211e/part-r-00000-0c84b915-c032-4f2e-abf5-1d48fdbddf38.snappy.parquet does not exist
```

This PR is to fix the issue.

### How was this patch tested?
Added a test case.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14523 from gatorsmile/refreshTempTable.
2016-08-08 22:34:28 +08:00
Nattavut Sutyanyong 06f5dc8415 [SPARK-16804][SQL] Correlated subqueries containing non-deterministic operations return incorrect results
## What changes were proposed in this pull request?

This patch fixes the incorrect results in the rule ResolveSubquery in Catalyst's Analysis phase by returning an error message when the LIMIT is found in the path from the parent table to the correlated predicate in the subquery.

## How was this patch tested?

./dev/run-tests
a new unit test on the problematic pattern.

Author: Nattavut Sutyanyong <nsy.can@gmail.com>

Closes #14411 from nsyca/master.
2016-08-08 12:14:11 +02:00
Weiqing Yang e10ca8de49 [SPARK-16945] Fix Java Lint errors
## What changes were proposed in this pull request?
This PR is to fix the minor Java linter errors as following:
[ERROR] src/main/java/org/apache/spark/sql/catalyst/expressions/VariableLengthRowBasedKeyValueBatch.java:[42,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/sql/catalyst/expressions/VariableLengthRowBasedKeyValueBatch.java:[97,10] (modifier) RedundantModifier: Redundant 'final' modifier.

## How was this patch tested?
Manual test.
dev/lint-java
Using `mvn` from path: /usr/local/bin/mvn
Checkstyle checks passed.

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #14532 from Sherry302/master.
2016-08-08 09:24:37 +01:00
Sean Owen 8d87252087 [SPARK-16409][SQL] regexp_extract with optional groups causes NPE
## What changes were proposed in this pull request?

regexp_extract actually returns null when it shouldn't when a regex matches but the requested optional group did not. This makes it return an empty string, as apparently designed.

## How was this patch tested?

Additional unit test

Author: Sean Owen <sowen@cloudera.com>

Closes #14504 from srowen/SPARK-16409.
2016-08-07 12:20:07 +01:00
Sylvain Zimmer 2460f03ffe [SPARK-16826][SQL] Switch to java.net.URI for parse_url()
## What changes were proposed in this pull request?
The java.net.URL class has a globally synchronized Hashtable, which limits the throughput of any single executor doing lots of calls to parse_url(). Tests have shown that a 36-core machine can only get to 10% CPU use because the threads are locked most of the time.

This patch switches to java.net.URI which has less features than java.net.URL but focuses on URI parsing, which is enough for parse_url().

New tests were added to make sure a few common edge cases didn't change behaviour.
https://issues.apache.org/jira/browse/SPARK-16826

## How was this patch tested?
I've kept the old URL code commented for now, so that people can verify that the new unit tests do pass with java.net.URL.

Thanks to srowen for the help!

Author: Sylvain Zimmer <sylvain@sylvainzimmer.com>

Closes #14488 from sylvinus/master.
2016-08-05 20:55:58 +01:00
Wenchen Fan 5effc016c8 [SPARK-16879][SQL] unify logical plans for CREATE TABLE and CTAS
## What changes were proposed in this pull request?

we have various logical plans for CREATE TABLE and CTAS: `CreateTableUsing`, `CreateTableUsingAsSelect`, `CreateHiveTableAsSelectLogicalPlan`. This PR unifies them to reduce the complexity and centralize the error handling.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14482 from cloud-fan/table.
2016-08-05 10:50:26 +02:00
Sean Zhong 9d7a47406e [SPARK-16853][SQL] fixes encoder error in DataSet typed select
## What changes were proposed in this pull request?

For DataSet typed select:
```
def select[U1: Encoder](c1: TypedColumn[T, U1]): Dataset[U1]
```
If type T is a case class or a tuple class that is not atomic, the resulting logical plan's schema will mismatch with `Dataset[T]` encoder's schema, which will cause encoder error and throw AnalysisException.

### Before change:
```
scala> case class A(a: Int, b: Int)
scala> Seq((0, A(1,2))).toDS.select($"_2".as[A])
org.apache.spark.sql.AnalysisException: cannot resolve '`a`' given input columns: [_2];
..
```

### After change:
```
scala> case class A(a: Int, b: Int)
scala> Seq((0, A(1,2))).toDS.select($"_2".as[A]).show
+---+---+
|  a|  b|
+---+---+
|  1|  2|
+---+---+
```

## How was this patch tested?

Unit test.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #14474 from clockfly/SPARK-16853.
2016-08-04 19:45:47 +08:00
Wenchen Fan 43f4fd6f9b [SPARK-16867][SQL] createTable and alterTable in ExternalCatalog should not take db
## What changes were proposed in this pull request?

These 2 methods take `CatalogTable` as parameter, which already have the database information.

## How was this patch tested?

existing test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14476 from cloud-fan/minor5.
2016-08-04 16:48:30 +08:00
Sean Zhong 27e815c31d [SPARK-16888][SQL] Implements eval method for expression AssertNotNull
## What changes were proposed in this pull request?

Implements `eval()` method for expression `AssertNotNull` so that we can convert local projection on LocalRelation to another LocalRelation.

### Before change:
```
scala> import org.apache.spark.sql.catalyst.dsl.expressions._
scala> import org.apache.spark.sql.catalyst.expressions.objects.AssertNotNull
scala> import org.apache.spark.sql.Column
scala> case class A(a: Int)
scala> Seq((A(1),2)).toDS().select(new Column(AssertNotNull("_1".attr, Nil))).explain

java.lang.UnsupportedOperationException: Only code-generated evaluation is supported.
  at org.apache.spark.sql.catalyst.expressions.objects.AssertNotNull.eval(objects.scala:850)
  ...
```

### After the change:
```
scala> Seq((A(1),2)).toDS().select(new Column(AssertNotNull("_1".attr, Nil))).explain(true)

== Parsed Logical Plan ==
'Project [assertnotnull('_1) AS assertnotnull(_1)#5]
+- LocalRelation [_1#2, _2#3]

== Analyzed Logical Plan ==
assertnotnull(_1): struct<a:int>
Project [assertnotnull(_1#2) AS assertnotnull(_1)#5]
+- LocalRelation [_1#2, _2#3]

== Optimized Logical Plan ==
LocalRelation [assertnotnull(_1)#5]

== Physical Plan ==
LocalTableScan [assertnotnull(_1)#5]
```

## How was this patch tested?

Unit test.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #14486 from clockfly/assertnotnull_eval.
2016-08-04 13:43:25 +08:00
Eric Liang e6f226c567 [SPARK-16596] [SQL] Refactor DataSourceScanExec to do partition discovery at execution instead of planning time
## What changes were proposed in this pull request?

Partition discovery is rather expensive, so we should do it at execution time instead of during physical planning. Right now there is not much benefit since ListingFileCatalog will read scan for all partitions at planning time anyways, but this can be optimized in the future. Also, there might be more information for partition pruning not available at planning time.

This PR moves a lot of the file scan logic from planning to execution time. All file scan operations are handled by `FileSourceScanExec`, which handles both batched and non-batched file scans. This requires some duplication with `RowDataSourceScanExec`, but is probably worth it so that `FileSourceScanExec` does not need to depend on an input RDD.

TODO: In another pr, move DataSourceScanExec to it's own file.

## How was this patch tested?

Existing tests (it might be worth adding a test that catalog.listFiles() is delayed until execution, but this can be delayed until there is an actual benefit to doing so).

Author: Eric Liang <ekl@databricks.com>

Closes #14241 from ericl/refactor.
2016-08-03 11:19:55 -07:00
Wenchen Fan b55f34370f [SPARK-16714][SPARK-16735][SPARK-16646] array, map, greatest, least's type coercion should handle decimal type
## What changes were proposed in this pull request?

Here is a table about the behaviours of `array`/`map` and `greatest`/`least` in Hive, MySQL and Postgres:

|    |Hive|MySQL|Postgres|
|---|---|---|---|---|
|`array`/`map`|can find a wider type with decimal type arguments, and will truncate the wider decimal type if necessary|can find a wider type with decimal type arguments, no truncation problem|can find a wider type with decimal type arguments, no truncation problem|
|`greatest`/`least`|can find a wider type with decimal type arguments, and truncate if necessary, but can't do string promotion|can find a wider type with decimal type arguments, no truncation problem, but can't do string promotion|can find a wider type with decimal type arguments, no truncation problem, but can't do string promotion|

I think these behaviours makes sense and Spark SQL should follow them.

This PR fixes `array` and `map` by using `findWiderCommonType` to get the wider type.
This PR fixes `greatest` and `least` by add a `findWiderTypeWithoutStringPromotion`, which provides similar semantic of `findWiderCommonType`, but without string promotion.

## How was this patch tested?

new tests in `TypeCoersionSuite`

Author: Wenchen Fan <wenchen@databricks.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #14439 from cloud-fan/bug.
2016-08-03 11:15:09 -07:00
Wenchen Fan a9beeaaaeb [SPARK-16855][SQL] move Greatest and Least from conditionalExpressions.scala to arithmetic.scala
## What changes were proposed in this pull request?

`Greatest` and `Least` are not conditional expressions, but arithmetic expressions.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14460 from cloud-fan/move.
2016-08-02 11:08:32 -07:00
Herman van Hovell 2330f3ecbb [SPARK-16836][SQL] Add support for CURRENT_DATE/CURRENT_TIMESTAMP literals
## What changes were proposed in this pull request?
In Spark 1.6 (with Hive support) we could use `CURRENT_DATE` and `CURRENT_TIMESTAMP` functions as literals (without adding braces), for example:
```SQL
select /* Spark 1.6: */ current_date, /* Spark 1.6  & Spark 2.0: */ current_date()
```
This was accidentally dropped in Spark 2.0. This PR reinstates this functionality.

## How was this patch tested?
Added a case to ExpressionParserSuite.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #14442 from hvanhovell/SPARK-16836.
2016-08-02 10:09:47 -07:00
Liang-Chi Hsieh 146001a9ff [SPARK-16062] [SPARK-15989] [SQL] Fix two bugs of Python-only UDTs
## What changes were proposed in this pull request?

There are two related bugs of Python-only UDTs. Because the test case of second one needs the first fix too. I put them into one PR. If it is not appropriate, please let me know.

### First bug: When MapObjects works on Python-only UDTs

`RowEncoder` will use `PythonUserDefinedType.sqlType` for its deserializer expression. If the sql type is `ArrayType`, we will have `MapObjects` working on it. But `MapObjects` doesn't consider `PythonUserDefinedType` as its input data type. It causes error like:

    import pyspark.sql.group
    from pyspark.sql.tests import PythonOnlyPoint, PythonOnlyUDT
    from pyspark.sql.types import *

    schema = StructType().add("key", LongType()).add("val", PythonOnlyUDT())
    df = spark.createDataFrame([(i % 3, PythonOnlyPoint(float(i), float(i))) for i in range(10)], schema=schema)
    df.show()

    File "/home/spark/python/lib/py4j-0.10.1-src.zip/py4j/protocol.py", line 312, in get_return_value py4j.protocol.Py4JJavaError: An error occurred while calling o36.showString.
    : java.lang.RuntimeException: Error while decoding: scala.MatchError: org.apache.spark.sql.types.PythonUserDefinedTypef4ceede8 (of class org.apache.spark.sql.types.PythonUserDefinedType)
    ...

### Second bug: When Python-only UDTs is the element type of ArrayType

    import pyspark.sql.group
    from pyspark.sql.tests import PythonOnlyPoint, PythonOnlyUDT
    from pyspark.sql.types import *

    schema = StructType().add("key", LongType()).add("val", ArrayType(PythonOnlyUDT()))
    df = spark.createDataFrame([(i % 3, [PythonOnlyPoint(float(i), float(i))]) for i in range(10)], schema=schema)
    df.show()

## How was this patch tested?
PySpark's sql tests.

Author: Liang-Chi Hsieh <simonh@tw.ibm.com>

Closes #13778 from viirya/fix-pyudt.
2016-08-02 10:08:18 -07:00
Tom Magrino 1dab63d8d3 [SPARK-16837][SQL] TimeWindow incorrectly drops slideDuration in constructors
## What changes were proposed in this pull request?

Fix of incorrect arguments (dropping slideDuration and using windowDuration) in constructors for TimeWindow.

The JIRA this addresses is here: https://issues.apache.org/jira/browse/SPARK-16837

## How was this patch tested?

Added a test to TimeWindowSuite to check that the results of TimeWindow object apply and TimeWindow class constructor are equivalent.

Author: Tom Magrino <tmagrino@fb.com>

Closes #14441 from tmagrino/windowing-fix.
2016-08-02 09:16:44 -07:00
petermaxlee a1ff72e1cc [SPARK-16850][SQL] Improve type checking error message for greatest/least
## What changes were proposed in this pull request?
Greatest/least function does not have the most friendly error message for data types. This patch improves the error message to not show the Seq type, and use more human readable data types.

Before:
```
org.apache.spark.sql.AnalysisException: cannot resolve 'greatest(CAST(1.0 AS DECIMAL(2,1)), "1.0")' due to data type mismatch: The expressions should all have the same type, got GREATEST (ArrayBuffer(DecimalType(2,1), StringType)).; line 1 pos 7
```

After:
```
org.apache.spark.sql.AnalysisException: cannot resolve 'greatest(CAST(1.0 AS DECIMAL(2,1)), "1.0")' due to data type mismatch: The expressions should all have the same type, got GREATEST(decimal(2,1), string).; line 1 pos 7
```

## How was this patch tested?
Manually verified the output and also added unit tests to ConditionalExpressionSuite.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14453 from petermaxlee/SPARK-16850.
2016-08-02 19:32:35 +08:00
Wenchen Fan 2eedc00b04 [SPARK-16828][SQL] remove MaxOf and MinOf
## What changes were proposed in this pull request?

These 2 expressions are not needed anymore after we have `Greatest` and `Least`. This PR removes them and related tests.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14434 from cloud-fan/minor1.
2016-08-01 17:54:41 -07:00
Holden Karau ab1e761f96 [SPARK-16774][SQL] Fix use of deprecated timestamp constructor & improve timezone handling
## What changes were proposed in this pull request?

Removes the deprecated timestamp constructor and incidentally fixes the use which was using system timezone rather than the one specified when working near DST.

This change also causes the roundtrip tests to fail since it now actually uses all the timezones near DST boundaries where it didn't before.

Note: this is only a partial the solution, longer term we should follow up with https://issues.apache.org/jira/browse/SPARK-16788 to avoid this problem & simplify our timezone handling code.

## How was this patch tested?

New tests for two timezones added so even if user timezone happens to coincided with one, the other tests should still fail. Important note: this (temporarily) disables the round trip tests until we can fix the issue more thoroughly.

Author: Holden Karau <holden@us.ibm.com>

Closes #14398 from holdenk/SPARK-16774-fix-use-of-deprecated-timestamp-constructor.
2016-08-01 13:57:05 -07:00
eyal farago 338a98d65c [SPARK-16791][SQL] cast struct with timestamp field fails
## What changes were proposed in this pull request?
a failing test case + fix to SPARK-16791 (https://issues.apache.org/jira/browse/SPARK-16791)

## How was this patch tested?
added a failing test case to CastSuit, then fixed the Cast code and rerun the entire CastSuit

Author: eyal farago <eyal farago>
Author: Eyal Farago <eyal.farago@actimize.com>

Closes #14400 from eyalfa/SPARK-16791_cast_struct_with_timestamp_field_fails.
2016-08-01 22:43:32 +08:00
Dongjoon Hyun 64d8f37c71 [SPARK-16726][SQL] Improve Union/Intersect/Except error messages on incompatible types
## What changes were proposed in this pull request?

Currently, `UNION` queries on incompatible types show misleading error messages, i.e., `unresolved operator Union`. We had better show a more correct message. This will help users in the situation of [SPARK-16704](https://issues.apache.org/jira/browse/SPARK-16704).

**Before**
```scala
scala> sql("select 1,2,3 union (select 1,array(2),3)")
org.apache.spark.sql.AnalysisException: unresolved operator 'Union;
scala> sql("select 1,2,3 intersect (select 1,array(2),3)")
org.apache.spark.sql.AnalysisException: unresolved operator 'Intersect;
scala> sql("select 1,2,3 except (select 1,array(2),3)")
org.apache.spark.sql.AnalysisException: unresolved operator 'Except;
```

**After**
```scala
scala> sql("select 1,2,3 union (select 1,array(2),3)")
org.apache.spark.sql.AnalysisException: Union can only be performed on tables with the compatible column types. ArrayType(IntegerType,false) <> IntegerType at the second column of the second table;
scala> sql("select 1,2,3 intersect (select 1,array(2),3)")
org.apache.spark.sql.AnalysisException: Intersect can only be performed on tables with the compatible column types. ArrayType(IntegerType,false) <> IntegerType at the second column of the second table;
scala> sql("select 1,2,3 except (select array(1),array(2),3)")
org.apache.spark.sql.AnalysisException: Except can only be performed on tables with the compatible column types. ArrayType(IntegerType,false) <> IntegerType at the first column of the second table;
```

## How was this patch tested?

Pass the Jenkins test with a new test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14355 from dongjoon-hyun/SPARK-16726.
2016-08-01 11:12:58 +02:00
Wenchen Fan 301fb0d723 [SPARK-16731][SQL] use StructType in CatalogTable and remove CatalogColumn
## What changes were proposed in this pull request?

`StructField` has very similar semantic with `CatalogColumn`, except that `CatalogColumn` use string to express data type. I think it's reasonable to use `StructType` as the `CatalogTable.schema` and remove `CatalogColumn`.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14363 from cloud-fan/column.
2016-07-31 18:18:53 -07:00
Reynold Xin 064d91ff73 [SPARK-16813][SQL] Remove private[sql] and private[spark] from catalyst package
## What changes were proposed in this pull request?
The catalyst package is meant to be internal, and as a result it does not make sense to mark things as private[sql] or private[spark]. It simply makes debugging harder when Spark developers need to inspect the plans at runtime.

This patch removes all private[sql] and private[spark] visibility modifiers in org.apache.spark.sql.catalyst.

## How was this patch tested?
N/A - just visibility changes.

Author: Reynold Xin <rxin@databricks.com>

Closes #14418 from rxin/SPARK-16813.
2016-07-31 16:31:06 +08:00
Sean Owen 0dc4310b47 [SPARK-16694][CORE] Use for/foreach rather than map for Unit expressions whose side effects are required
## What changes were proposed in this pull request?

Use foreach/for instead of map where operation requires execution of body, not actually defining a transformation

## How was this patch tested?

Jenkins

Author: Sean Owen <sowen@cloudera.com>

Closes #14332 from srowen/SPARK-16694.
2016-07-30 04:42:38 -07:00
Tathagata Das bbc247548a [SPARK-16748][SQL] SparkExceptions during planning should not wrapped in TreeNodeException
## What changes were proposed in this pull request?
We do not want SparkExceptions from job failures in the planning phase to create TreeNodeException. Hence do not wrap SparkException in TreeNodeException.

## How was this patch tested?
New unit test

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #14395 from tdas/SPARK-16748.
2016-07-29 19:59:35 -07:00
Liang-Chi Hsieh 9ade77c3fa [SPARK-16639][SQL] The query with having condition that contains grouping by column should work
## What changes were proposed in this pull request?

The query with having condition that contains grouping by column will be failed during analysis. E.g.,

    create table tbl(a int, b string);
    select count(b) from tbl group by a + 1 having a + 1 = 2;

Having condition should be able to use grouping by column.

## How was this patch tested?

Jenkins tests.

Author: Liang-Chi Hsieh <simonh@tw.ibm.com>

Closes #14296 from viirya/having-contains-grouping-column.
2016-07-28 22:33:33 +08:00
petermaxlee 11d427c924 [SPARK-16730][SQL] Implement function aliases for type casts
## What changes were proposed in this pull request?
Spark 1.x supports using the Hive type name as function names for doing casts, e.g.
```sql
SELECT int(1.0);
SELECT string(2.0);
```

The above query would work in Spark 1.x because Spark 1.x fail back to Hive for unimplemented functions, and break in Spark 2.0 because the fall back was removed.

This patch implements function aliases using an analyzer rule for the following cast functions:
- boolean
- tinyint
- smallint
- int
- bigint
- float
- double
- decimal
- date
- timestamp
- binary
- string

## How was this patch tested?
Added end-to-end tests in SQLCompatibilityFunctionSuite.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14364 from petermaxlee/SPARK-16730-2.
2016-07-28 13:13:17 +08:00
petermaxlee ef0ccbcb07 [SPARK-16729][SQL] Throw analysis exception for invalid date casts
## What changes were proposed in this pull request?
Spark currently throws exceptions for invalid casts for all other data types except date type. Somehow date type returns null. It should be consistent and throws analysis exception as well.

## How was this patch tested?
Added a unit test case in CastSuite.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14358 from petermaxlee/SPARK-16729.
2016-07-27 16:04:43 +08:00
Qifan Pu 738b4cc548 [SPARK-16524][SQL] Add RowBatch and RowBasedHashMapGenerator
## What changes were proposed in this pull request?

This PR is the first step for the following feature:

For hash aggregation in Spark SQL, we use a fast aggregation hashmap to act as a "cache" in order to boost aggregation performance. Previously, the hashmap is backed by a `ColumnarBatch`. This has performance issues when we have wide schema for the aggregation table (large number of key fields or value fields).
In this JIRA, we support another implementation of fast hashmap, which is backed by a `RowBasedKeyValueBatch`. We then automatically pick between the two implementations based on certain knobs.

In this first-step PR, implementations for `RowBasedKeyValueBatch` and `RowBasedHashMapGenerator` are added.

## How was this patch tested?

Unit tests: `RowBasedKeyValueBatchSuite`

Author: Qifan Pu <qifan.pu@gmail.com>

Closes #14349 from ooq/SPARK-16524.
2016-07-26 18:08:07 -07:00
Wenchen Fan 6959061f02 [SPARK-16706][SQL] support java map in encoder
## What changes were proposed in this pull request?

finish the TODO, create a new expression `ExternalMapToCatalyst` to iterate the map directly.

## How was this patch tested?

new test in `JavaDatasetSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14344 from cloud-fan/java-map.
2016-07-26 15:33:05 +08:00
Liang-Chi Hsieh 7b06a8948f [SPARK-16686][SQL] Remove PushProjectThroughSample since it is handled by ColumnPruning
## What changes were proposed in this pull request?

We push down `Project` through `Sample` in `Optimizer` by the rule `PushProjectThroughSample`. However, if the projected columns produce new output, they will encounter whole data instead of sampled data. It will bring some inconsistency between original plan (Sample then Project) and optimized plan (Project then Sample). In the extreme case such as attached in the JIRA, if the projected column is an UDF which is supposed to not see the sampled out data, the result of UDF will be incorrect.

Since the rule `ColumnPruning` already handles general `Project` pushdown. We don't need  `PushProjectThroughSample` anymore. The rule `ColumnPruning` also avoids the described issue.

## How was this patch tested?

Jenkins tests.

Author: Liang-Chi Hsieh <simonh@tw.ibm.com>

Closes #14327 from viirya/fix-sample-pushdown.
2016-07-26 12:00:01 +08:00
Yin Huai 815f3eece5 [SPARK-16633][SPARK-16642][SPARK-16721][SQL] Fixes three issues related to lead and lag functions
## What changes were proposed in this pull request?
This PR contains three changes.

First, this PR changes the behavior of lead/lag back to Spark 1.6's behavior, which is described as below:
1. lead/lag respect null input values, which means that if the offset row exists and the input value is null, the result will be null instead of the default value.
2. If the offset row does not exist, the default value will be used.
3. OffsetWindowFunction's nullable setting also considers the nullability of its input (because of the first change).

Second, this PR fixes the evaluation of lead/lag when the input expression is a literal. This fix is a result of the first change. In current master, if a literal is used as the input expression of a lead or lag function, the result will be this literal even if the offset row does not exist.

Third, this PR makes ResolveWindowFrame not fire if a window function is not resolved.

## How was this patch tested?
New tests in SQLWindowFunctionSuite

Author: Yin Huai <yhuai@databricks.com>

Closes #14284 from yhuai/lead-lag.
2016-07-25 20:58:07 -07:00
Michael Armbrust f99e34e8e5 [SPARK-16724] Expose DefinedByConstructorParams
We don't generally make things in catalyst/execution private.  Instead they are just undocumented due to their lack of stability guarantees.

Author: Michael Armbrust <michael@databricks.com>

Closes #14356 from marmbrus/patch-1.
2016-07-25 20:41:24 -07:00
gatorsmile 3fc4566941 [SPARK-16678][SPARK-16677][SQL] Fix two View-related bugs
## What changes were proposed in this pull request?
**Issue 1: Disallow Creating/Altering a View when the same-name Table Exists (without IF NOT EXISTS)**
When we create OR alter a view, we check whether the view already exists. In the current implementation, if a table with the same name exists, we treat it as a view. However, this is not the right behavior. We should follow what Hive does. For example,
```
hive> CREATE TABLE tab1 (id int);
OK
Time taken: 0.196 seconds
hive> CREATE OR REPLACE VIEW tab1 AS SELECT * FROM t1;
FAILED: SemanticException [Error 10218]: Existing table is not a view
 The following is an existing table, not a view: default.tab1
hive> ALTER VIEW tab1 AS SELECT * FROM t1;
FAILED: SemanticException [Error 10218]: Existing table is not a view
 The following is an existing table, not a view: default.tab1
hive> CREATE VIEW IF NOT EXISTS tab1 AS SELECT * FROM t1;
OK
Time taken: 0.678 seconds
```

**Issue 2: Strange Error when Issuing Load Table Against A View**
Users should not be allowed to issue LOAD DATA against a view. Currently, when users doing it, we got a very strange runtime error. For example,
```SQL
LOAD DATA LOCAL INPATH "$testData" INTO TABLE $viewName
```
```
java.lang.reflect.InvocationTargetException was thrown.
java.lang.reflect.InvocationTargetException
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:606)
	at org.apache.spark.sql.hive.client.Shim_v0_14.loadTable(HiveShim.scala:680)
```
## How was this patch tested?
Added test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14314 from gatorsmile/tableDDLAgainstView.
2016-07-26 09:32:29 +08:00
Shixiong Zhu 12f490b5c8 [SPARK-16715][TESTS] Fix a potential ExprId conflict for SubexpressionEliminationSuite."Semantic equals and hash"
## What changes were proposed in this pull request?

SubexpressionEliminationSuite."Semantic equals and hash" assumes the default AttributeReference's exprId wont' be "ExprId(1)". However, that depends on when this test runs. It may happen to use "ExprId(1)".

This PR detects the conflict and makes sure we create a different ExprId when the conflict happens.

## How was this patch tested?

Jenkins unit tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #14350 from zsxwing/SPARK-16715.
2016-07-25 16:08:29 -07:00
Cheng Lian 7ea6d282b9 [SPARK-16703][SQL] Remove extra whitespace in SQL generation for window functions
## What changes were proposed in this pull request?

This PR fixes a minor formatting issue of `WindowSpecDefinition.sql` when no partitioning expressions are present.

Before:

```sql
( ORDER BY `a` ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)
```

After:

```sql
(ORDER BY `a` ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)
```

## How was this patch tested?

New test case added in `ExpressionSQLBuilderSuite`.

Author: Cheng Lian <lian@databricks.com>

Closes #14334 from liancheng/window-spec-sql-format.
2016-07-25 09:42:39 -07:00
hyukjinkwon 79826f3c79 [SPARK-16698][SQL] Field names having dots should be allowed for datasources based on FileFormat
## What changes were proposed in this pull request?

It seems this is a regression assuming from https://issues.apache.org/jira/browse/SPARK-16698.

Field name having dots throws an exception. For example the codes below:

```scala
val path = "/tmp/path"
val json =""" {"a.b":"data"}"""
spark.sparkContext
  .parallelize(json :: Nil)
  .saveAsTextFile(path)
spark.read.json(path).collect()
```

throws an exception as below:

```
Unable to resolve a.b given [a.b];
org.apache.spark.sql.AnalysisException: Unable to resolve a.b given [a.b];
	at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1$$anonfun$apply$5.apply(LogicalPlan.scala:134)
	at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1$$anonfun$apply$5.apply(LogicalPlan.scala:134)
	at scala.Option.getOrElse(Option.scala:121)
```

This problem was introduced in 17eec0a71b (diff-27c76f96a7b2733ecfd6f46a1716e153R121)

When extracting the data columns, it does not count that it can contains dots in field names. Actually, it seems the fields name are not expected as quoted when defining schema. So, It not have to consider whether this is wrapped with quotes because the actual schema (inferred or user-given schema) would not have the quotes for fields.

For example, this throws an exception. (**Loading JSON from RDD is fine**)

```scala
val json =""" {"a.b":"data"}"""
val rdd = spark.sparkContext.parallelize(json :: Nil)
spark.read.schema(StructType(Seq(StructField("`a.b`", StringType, true))))
  .json(rdd).select("`a.b`").printSchema()
```

as below:

```
cannot resolve '```a.b```' given input columns: [`a.b`];
org.apache.spark.sql.AnalysisException: cannot resolve '```a.b```' given input columns: [`a.b`];
	at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
```

## How was this patch tested?

Unit tests in `FileSourceStrategySuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14339 from HyukjinKwon/SPARK-16698-regression.
2016-07-25 22:51:30 +08:00
Wenchen Fan 64529b186a [SPARK-16691][SQL] move BucketSpec to catalyst module and use it in CatalogTable
## What changes were proposed in this pull request?

It's weird that we have `BucketSpec` to abstract bucket info, but don't use it in `CatalogTable`. This PR moves `BucketSpec` into catalyst module.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14331 from cloud-fan/check.
2016-07-25 22:05:48 +08:00
Wenchen Fan d27d362eba [SPARK-16660][SQL] CreateViewCommand should not take CatalogTable
## What changes were proposed in this pull request?

`CreateViewCommand` only needs some information of a `CatalogTable`, but not all of them. We have some tricks(e.g. we need to check the table type is `VIEW`, we need to make `CatalogColumn.dataType` nullable) to allow it to take a `CatalogTable`.
This PR cleans it up and only pass in necessary information to `CreateViewCommand`.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14297 from cloud-fan/minor2.
2016-07-25 22:02:00 +08:00
Cheng Lian 68b4020d0c [SPARK-16648][SQL] Make ignoreNullsExpr a child expression of First and Last
## What changes were proposed in this pull request?

Default `TreeNode.withNewChildren` implementation doesn't work for `Last` and when both constructor arguments are the same, e.g.:

```sql
LAST_VALUE(FALSE) -- The 2nd argument defaults to FALSE
LAST_VALUE(FALSE, FALSE)
LAST_VALUE(TRUE, TRUE)
```

This is because although `Last` is a unary expression, both of its constructor arguments, `child` and `ignoreNullsExpr`, are `Expression`s. When they have the same value, `TreeNode.withNewChildren` treats both of them as child nodes by mistake. `First` is also affected by this issue in exactly the same way.

This PR fixes this issue by making `ignoreNullsExpr` a child expression of `First` and `Last`.

## How was this patch tested?

New test case added in `WindowQuerySuite`.

Author: Cheng Lian <lian@databricks.com>

Closes #14295 from liancheng/spark-16648-last-value.
2016-07-25 17:22:29 +08:00
Wenchen Fan 1221ce0402 [SPARK-16645][SQL] rename CatalogStorageFormat.serdeProperties to properties
## What changes were proposed in this pull request?

we also store data source table options in this field, it's unreasonable to call it `serdeProperties`.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14283 from cloud-fan/minor1.
2016-07-25 09:28:56 +08:00
Liang-Chi Hsieh e10b8741d8 [SPARK-16622][SQL] Fix NullPointerException when the returned value of the called method in Invoke is null
## What changes were proposed in this pull request?

Currently we don't check the value returned by called method in `Invoke`. When the returned value is null and is assigned to a variable of primitive type, `NullPointerException` will be thrown.

## How was this patch tested?

Jenkins tests.

Author: Liang-Chi Hsieh <simonh@tw.ibm.com>

Closes #14259 from viirya/agg-empty-ds.
2016-07-23 10:27:16 +08:00
Jacek Laskowski e1bd70f44b [SPARK-16287][HOTFIX][BUILD][SQL] Fix annotation argument needs to be a constant
## What changes were proposed in this pull request?

Build fix for [SPARK-16287][SQL] Implement str_to_map SQL function that has introduced this compilation error:

```
/Users/jacek/dev/oss/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala:402: error: annotation argument needs to be a constant; found: "_FUNC_(text[, pairDelim, keyValueDelim]) - Creates a map after splitting the text ".+("into key/value pairs using delimiters. ").+("Default delimiters are \',\' for pairDelim and \':\' for keyValueDelim.")
    "into key/value pairs using delimiters. " +
                                              ^
```

## How was this patch tested?

Local build

Author: Jacek Laskowski <jacek@japila.pl>

Closes #14315 from jaceklaskowski/build-fix-complexTypeCreator.
2016-07-22 12:37:30 +01:00
Sandeep Singh df2c6d59d0 [SPARK-16287][SQL] Implement str_to_map SQL function
## What changes were proposed in this pull request?
This PR adds `str_to_map` SQL function in order to remove Hive fallback.

## How was this patch tested?
Pass the Jenkins tests with newly added.

Author: Sandeep Singh <sandeep@techaddict.me>

Closes #13990 from techaddict/SPARK-16287.
2016-07-22 10:05:21 +08:00
Liang-Chi Hsieh 6203668d50 [SPARK-16640][SQL] Add codegen for Elt function
## What changes were proposed in this pull request?

Elt function doesn't support codegen execution now. We should add the support.

## How was this patch tested?

Jenkins tests.

Author: Liang-Chi Hsieh <simonh@tw.ibm.com>

Closes #14277 from viirya/elt-codegen.
2016-07-21 20:54:17 +08:00
Wenchen Fan cfa5ae84ed [SPARK-16644][SQL] Aggregate should not propagate constraints containing aggregate expressions
## What changes were proposed in this pull request?

aggregate expressions can only be executed inside `Aggregate`, if we propagate it up with constraints, the parent operator can not execute it and will fail at runtime.

## How was this patch tested?

new test in SQLQuerySuite

Author: Wenchen Fan <wenchen@databricks.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #14281 from cloud-fan/bug.
2016-07-20 18:37:15 -07:00
Marcelo Vanzin e3cd5b3050 [SPARK-16634][SQL] Workaround JVM bug by moving some code out of ctor.
Some 1.7 JVMs have a bug that is triggered by certain Scala-generated
bytecode. GenericArrayData suffers from that and fails to load in certain
JVMs.

Moving the offending code out of the constructor and into a helper method
avoids the issue.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #14271 from vanzin/SPARK-16634.
2016-07-20 10:38:44 -07:00
Dongjoon Hyun 162d04a30e [SPARK-16602][SQL] Nvl function should support numeric-string cases
## What changes were proposed in this pull request?

`Nvl` function should support numeric-straing cases like Hive/Spark1.6. Currently, `Nvl` finds the tightest common types among numeric types. This PR extends that to consider `String` type, too.

```scala
- TypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype =>
+ TypeCoercion.findTightestCommonTypeToString(left.dataType, right.dataType).map { dtype =>
```

**Before**
```scala
scala> sql("select nvl('0', 1)").collect()
org.apache.spark.sql.AnalysisException: cannot resolve `nvl("0", 1)` due to data type mismatch:
input to function coalesce should all be the same type, but it's [string, int]; line 1 pos 7
```

**After**
```scala
scala> sql("select nvl('0', 1)").collect()
res0: Array[org.apache.spark.sql.Row] = Array([0])
```

## How was this patch tested?

Pass the Jenkins tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14251 from dongjoon-hyun/SPARK-16602.
2016-07-19 10:28:17 -07:00
Xin Ren 21a6dd2aef [SPARK-16535][BUILD] In pom.xml, remove groupId which is redundant definition and inherited from the parent
https://issues.apache.org/jira/browse/SPARK-16535

## What changes were proposed in this pull request?

When I scan through the pom.xml of sub projects, I found this warning as below and attached screenshot
```
Definition of groupId is redundant, because it's inherited from the parent
```
![screen shot 2016-07-13 at 3 13 11 pm](https://cloud.githubusercontent.com/assets/3925641/16823121/744f893e-4916-11e6-8a52-042f83b9db4e.png)

I've tried to remove some of the lines with groupId definition, and the build on my local machine is still ok.
```
<groupId>org.apache.spark</groupId>
```
As I just find now `<maven.version>3.3.9</maven.version>` is being used in Spark 2.x, and Maven-3 supports versionless parent elements: Maven 3 will remove the need to specify the parent version in sub modules. THIS is great (in Maven 3.1).

ref: http://stackoverflow.com/questions/3157240/maven-3-worth-it/3166762#3166762

## How was this patch tested?

I've tested by re-building the project, and build succeeded.

Author: Xin Ren <iamshrek@126.com>

Closes #14189 from keypointt/SPARK-16535.
2016-07-19 11:59:46 +01:00
Reynold Xin 7b84758034 [SPARK-16584][SQL] Move regexp unit tests to RegexpExpressionsSuite
## What changes were proposed in this pull request?
This patch moves regexp related unit tests from StringExpressionsSuite to RegexpExpressionsSuite to match the file name for regexp expressions.

## How was this patch tested?
This is a test only change.

Author: Reynold Xin <rxin@databricks.com>

Closes #14230 from rxin/SPARK-16584.
2016-07-16 23:42:28 -07:00
Sameer Agarwal a1ffbada8a [SPARK-16582][SQL] Explicitly define isNull = false for non-nullable expressions
## What changes were proposed in this pull request?

This patch is just a slightly safer way to fix the issue we encountered in https://github.com/apache/spark/pull/14168 should this pattern re-occur at other places in the code.

## How was this patch tested?

Existing tests. Also, I manually tested that it fixes the problem in SPARK-16514 without having the proposed change in https://github.com/apache/spark/pull/14168

Author: Sameer Agarwal <sameerag@cs.berkeley.edu>

Closes #14227 from sameeragarwal/codegen.
2016-07-16 13:24:00 -07:00
gatorsmile 1b5c9e52a7 [SPARK-16530][SQL][TRIVIAL] Wrong Parser Keyword in ALTER TABLE CHANGE COLUMN
#### What changes were proposed in this pull request?
Based on the [Hive SQL syntax](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-ChangeColumnName/Type/Position/Comment), the command to change column name/type/position/comment is `ALTER TABLE CHANGE COLUMN`. However, in our .g4 file, it is `ALTER TABLE CHANGE COLUMNS`. Because it is the last optional keyword, it does not take any effect. Thus, I put the issue as a Trivial level.

cc hvanhovell

#### How was this patch tested?
Existing test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14186 from gatorsmile/changeColumns.
2016-07-14 17:15:51 +02:00
Wenchen Fan db7317ac3c [SPARK-16448] RemoveAliasOnlyProject should not remove alias with metadata
## What changes were proposed in this pull request?

`Alias` with metadata is not a no-op and we should not strip it in `RemoveAliasOnlyProject` rule.
This PR also did some improvement for this rule:

1. extend the semantic of `alias-only`. Now we allow the project list to be partially aliased.
2. add unit test for this rule.

## How was this patch tested?

new `RemoveAliasOnlyProjectSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14106 from cloud-fan/bug.
2016-07-14 15:48:22 +08:00
蒋星博 f376c37268 [SPARK-16343][SQL] Improve the PushDownPredicate rule to pushdown predicates correctly in non-deterministic condition.
## What changes were proposed in this pull request?

Currently our Optimizer may reorder the predicates to run them more efficient, but in non-deterministic condition, change the order between deterministic parts and non-deterministic parts may change the number of input rows. For example:
```SELECT a FROM t WHERE rand() < 0.1 AND a = 1```
And
```SELECT a FROM t WHERE a = 1 AND rand() < 0.1```
may call rand() for different times and therefore the output rows differ.

This PR improved this condition by checking whether the predicate is placed before any non-deterministic predicates.

## How was this patch tested?

Expanded related testcases in FilterPushdownSuite.

Author: 蒋星博 <jiangxingbo@meituan.com>

Closes #14012 from jiangxb1987/ppd.
2016-07-14 00:21:27 +08:00
Eric Liang 1c58fa905b [SPARK-16514][SQL] Fix various regex codegen bugs
## What changes were proposed in this pull request?

RegexExtract and RegexReplace currently crash on non-nullable input due use of a hard-coded local variable name (e.g. compiles fail with `java.lang.Exception: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 85, Column 26: Redefinition of local variable "m" `).

This changes those variables to use fresh names, and also in a few other places.

## How was this patch tested?

Unit tests. rxin

Author: Eric Liang <ekl@databricks.com>

Closes #14168 from ericl/sc-3906.
2016-07-12 23:09:02 -07:00
petermaxlee 56bd399a86 [SPARK-16284][SQL] Implement reflect SQL function
## What changes were proposed in this pull request?
This patch implements reflect SQL function, which can be used to invoke a Java method in SQL. Slightly different from Hive, this implementation requires the class name and the method name to be literals. This implementation also supports only a smaller number of data types, and requires the function to be static, as suggested by rxin in #13969.

java_method is an alias for reflect, so this should also resolve SPARK-16277.

## How was this patch tested?
Added expression unit tests and an end-to-end test.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14138 from petermaxlee/reflect-static.
2016-07-13 08:05:20 +08:00
Marcelo Vanzin 7f968867ff [SPARK-16119][SQL] Support PURGE option to drop table / partition.
This option is used by Hive to directly delete the files instead of
moving them to the trash. This is needed in certain configurations
where moving the files does not work. For non-Hive tables and partitions,
Spark already behaves as if the PURGE option was set, so there's no
need to do anything.

Hive support for PURGE was added in 0.14 (for tables) and 1.2 (for
partitions), so the code reflects that: trying to use the option with
older versions of Hive will cause an exception to be thrown.

The change is a little noisier than I would like, because of the code
to propagate the new flag through all the interfaces and implementations;
the main changes are in the parser and in HiveShim, aside from the tests
(DDLCommandSuite, VersionsSuite).

Tested by running sql and catalyst unit tests, plus VersionsSuite which
has been updated to test the version-specific behavior. I also ran an
internal test suite that uses PURGE and would not pass previously.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #13831 from vanzin/SPARK-16119.
2016-07-12 12:47:46 -07:00
Reynold Xin c377e49e38 [SPARK-16489][SQL] Guard against variable reuse mistakes in expression code generation
## What changes were proposed in this pull request?
In code generation, it is incorrect for expressions to reuse variable names across different instances of itself. As an example, SPARK-16488 reports a bug in which pmod expression reuses variable name "r".

This patch updates ExpressionEvalHelper test harness to always project two instances of the same expression, which will help us catch variable reuse problems in expression unit tests. This patch also fixes the bug in crc32 expression.

## How was this patch tested?
This is a test harness change, but I also created a new test suite for testing the test harness.

Author: Reynold Xin <rxin@databricks.com>

Closes #14146 from rxin/SPARK-16489.
2016-07-12 10:07:23 -07:00
Sameer Agarwal 9cc74f95ed [SPARK-16488] Fix codegen variable namespace collision in pmod and partitionBy
## What changes were proposed in this pull request?

This patch fixes a variable namespace collision bug in pmod and partitionBy

## How was this patch tested?

Regression test for one possible occurrence. A more general fix in `ExpressionEvalHelper.checkEvaluation` will be in a subsequent PR.

Author: Sameer Agarwal <sameer@databricks.com>

Closes #14144 from sameeragarwal/codegen-bug.
2016-07-11 20:26:01 -07:00
Dongjoon Hyun 840853ed06 [SPARK-16458][SQL] SessionCatalog should support listColumns for temporary tables
## What changes were proposed in this pull request?

Temporary tables are used frequently, but `spark.catalog.listColumns` does not support those tables. This PR make `SessionCatalog` supports temporary table column listing.

**Before**
```scala
scala> spark.range(10).createOrReplaceTempView("t1")

scala> spark.catalog.listTables().collect()
res1: Array[org.apache.spark.sql.catalog.Table] = Array(Table[name=`t1`, tableType=`TEMPORARY`, isTemporary=`true`])

scala> spark.catalog.listColumns("t1").collect()
org.apache.spark.sql.AnalysisException: Table `t1` does not exist in database `default`.;
```

**After**
```
scala> spark.catalog.listColumns("t1").collect()
res2: Array[org.apache.spark.sql.catalog.Column] = Array(Column[name='id', description='id', dataType='bigint', nullable='false', isPartition='false', isBucket='false'])
```
## How was this patch tested?

Pass the Jenkins tests including a new testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14114 from dongjoon-hyun/SPARK-16458.
2016-07-11 22:45:22 +02:00
Reynold Xin ffcb6e055a [SPARK-16477] Bump master version to 2.1.0-SNAPSHOT
## What changes were proposed in this pull request?
After SPARK-16476 (committed earlier today as #14128), we can finally bump the version number.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #14130 from rxin/SPARK-16477.
2016-07-11 09:42:56 -07:00
Dongjoon Hyun 7ac79da0e4 [SPARK-16459][SQL] Prevent dropping current database
## What changes were proposed in this pull request?

This PR prevents dropping the current database to avoid errors like the followings.

```scala
scala> sql("create database delete_db")
scala> sql("use delete_db")
scala> sql("drop database delete_db")
scala> sql("create table t as select 1")
org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException: Database `delete_db` not found;
```

## How was this patch tested?

Pass the Jenkins tests including an updated testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14115 from dongjoon-hyun/SPARK-16459.
2016-07-11 15:15:47 +02:00
gatorsmile e226278941 [SPARK-16355][SPARK-16354][SQL] Fix Bugs When LIMIT/TABLESAMPLE is Non-foldable, Zero or Negative
#### What changes were proposed in this pull request?
**Issue 1:** When a query containing LIMIT/TABLESAMPLE 0, the statistics could be zero. Results are correct but it could cause a huge performance regression. For example,
```Scala
Seq(("one", 1), ("two", 2), ("three", 3), ("four", 4)).toDF("k", "v")
  .createOrReplaceTempView("test")
val df1 = spark.table("test")
val df2 = spark.table("test").limit(0)
val df = df1.join(df2, Seq("k"), "left")
```
The statistics of both `df` and `df2` are zero. The statistics values should never be zero; otherwise `sizeInBytes` of `BinaryNode` will also be zero (product of children). This PR is to increase it to `1` when the num of rows is equal to 0.

**Issue 2:** When a query containing negative LIMIT/TABLESAMPLE, we should issue exceptions. Negative values could break the implementation assumption of multiple parts. For example, statistics calculation.  Below is the example query.
```SQL
SELECT * FROM testData TABLESAMPLE (-1 rows)
SELECT * FROM testData LIMIT -1
```
This PR is to issue an appropriate exception in this case.

**Issue 3:** Spark SQL follows the restriction of LIMIT clause in Hive. The argument to the LIMIT clause must evaluate to a constant value. It can be a numeric literal, or another kind of numeric expression involving operators, casts, and function return values. You cannot refer to a column or use a subquery. Currently, we do not detect whether the expression in LIMIT clause is foldable or not. If non-foldable, we might issue a strange error message. For example,
```SQL
SELECT * FROM testData LIMIT rand() > 0.2
```
Then, a misleading error message is issued, like
```
assertion failed: No plan for GlobalLimit (_nondeterministic#203 > 0.2)
+- Project [key#11, value#12, rand(-1441968339187861415) AS _nondeterministic#203]
   +- LocalLimit (_nondeterministic#202 > 0.2)
      +- Project [key#11, value#12, rand(-1308350387169017676) AS _nondeterministic#202]
         +- LogicalRDD [key#11, value#12]

java.lang.AssertionError: assertion failed: No plan for GlobalLimit (_nondeterministic#203 > 0.2)
+- Project [key#11, value#12, rand(-1441968339187861415) AS _nondeterministic#203]
   +- LocalLimit (_nondeterministic#202 > 0.2)
      +- Project [key#11, value#12, rand(-1308350387169017676) AS _nondeterministic#202]
         +- LogicalRDD [key#11, value#12]
```
This PR detects it and then issues a meaningful error message.

#### How was this patch tested?
Added test cases.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14034 from gatorsmile/limit.
2016-07-11 16:21:13 +08:00
petermaxlee 82f0874453 [SPARK-16318][SQL] Implement all remaining xpath functions
## What changes were proposed in this pull request?
This patch implements all remaining xpath functions that Hive supports and not natively supported in Spark: xpath_int, xpath_short, xpath_long, xpath_float, xpath_double, xpath_string, and xpath.

## How was this patch tested?
Added unit tests and end-to-end tests.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #13991 from petermaxlee/SPARK-16318.
2016-07-11 13:28:34 +08:00
wujian f5fef69143 [SPARK-16281][SQL] Implement parse_url SQL function
## What changes were proposed in this pull request?

This PR adds parse_url SQL functions in order to remove Hive fallback.

A new implementation of #13999

## How was this patch tested?

Pass the exist tests including new testcases.

Author: wujian <jan.chou.wu@gmail.com>

Closes #14008 from janplus/SPARK-16281.
2016-07-08 14:38:05 -07:00
Dongjoon Hyun a54438cb23 [SPARK-16285][SQL] Implement sentences SQL functions
## What changes were proposed in this pull request?

This PR implements `sentences` SQL function.

## How was this patch tested?

Pass the Jenkins tests with a new testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14004 from dongjoon-hyun/SPARK_16285.
2016-07-08 17:05:24 +08:00
petermaxlee 8228b06303 [SPARK-16436][SQL] checkEvaluation should support NaN
## What changes were proposed in this pull request?
This small patch modifies ExpressionEvalHelper. checkEvaluation to support comparing NaN values for floating point comparisons.

## How was this patch tested?
This is a test harness change.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14103 from petermaxlee/SPARK-16436.
2016-07-08 16:49:02 +08:00
Dongjoon Hyun dff73bfa5e [SPARK-16052][SQL] Improve CollapseRepartition optimizer for Repartition/RepartitionBy
## What changes were proposed in this pull request?

This PR improves `CollapseRepartition` to optimize the adjacent combinations of **Repartition** and **RepartitionBy**. Also, this PR adds a testsuite for this optimizer.

**Target Scenario**
```scala
scala> val dsView1 = spark.range(8).repartition(8, $"id")
scala> dsView1.createOrReplaceTempView("dsView1")
scala> sql("select id from dsView1 distribute by id").explain(true)
```

**Before**
```scala
scala> sql("select id from dsView1 distribute by id").explain(true)
== Parsed Logical Plan ==
'RepartitionByExpression ['id]
+- 'Project ['id]
   +- 'UnresolvedRelation `dsView1`

== Analyzed Logical Plan ==
id: bigint
RepartitionByExpression [id#0L]
+- Project [id#0L]
   +- SubqueryAlias dsview1
      +- RepartitionByExpression [id#0L], 8
         +- Range (0, 8, splits=8)

== Optimized Logical Plan ==
RepartitionByExpression [id#0L]
+- RepartitionByExpression [id#0L], 8
   +- Range (0, 8, splits=8)

== Physical Plan ==
Exchange hashpartitioning(id#0L, 200)
+- Exchange hashpartitioning(id#0L, 8)
   +- *Range (0, 8, splits=8)
```

**After**
```scala
scala> sql("select id from dsView1 distribute by id").explain(true)
== Parsed Logical Plan ==
'RepartitionByExpression ['id]
+- 'Project ['id]
   +- 'UnresolvedRelation `dsView1`

== Analyzed Logical Plan ==
id: bigint
RepartitionByExpression [id#0L]
+- Project [id#0L]
   +- SubqueryAlias dsview1
      +- RepartitionByExpression [id#0L], 8
         +- Range (0, 8, splits=8)

== Optimized Logical Plan ==
RepartitionByExpression [id#0L]
+- Range (0, 8, splits=8)

== Physical Plan ==
Exchange hashpartitioning(id#0L, 200)
+- *Range (0, 8, splits=8)
```

## How was this patch tested?

Pass the Jenkins tests (including a new testsuite).

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13765 from dongjoon-hyun/SPARK-16052.
2016-07-08 16:44:53 +08:00
Daoyuan Wang 28710b42b0 [SPARK-16415][SQL] fix catalog string error
## What changes were proposed in this pull request?

In #13537 we truncate `simpleString` if it is a long `StructType`. But sometimes we need `catalogString` to reconstruct `TypeInfo`, for example in description of [SPARK-16415 ](https://issues.apache.org/jira/browse/SPARK-16415). So we need to keep the implementation of `catalogString` not affected by our truncate.

## How was this patch tested?

added a test case.

Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #14089 from adrian-wang/catalogstring.
2016-07-07 11:08:06 -07:00
Dongjoon Hyun a04cab8f17 [SPARK-16174][SQL] Improve OptimizeIn optimizer to remove literal repetitions
## What changes were proposed in this pull request?

This PR improves `OptimizeIn` optimizer to remove the literal repetitions from SQL `IN` predicates. This optimizer prevents user mistakes and also can optimize some queries like [TPCDS-36](https://github.com/apache/spark/blob/master/sql/core/src/test/resources/tpcds/q36.sql#L19).

**Before**
```scala
scala> sql("select state from (select explode(array('CA','TN')) state) where state in ('TN','TN','TN','TN','TN','TN','TN')").explain
== Physical Plan ==
*Filter state#6 IN (TN,TN,TN,TN,TN,TN,TN)
+- Generate explode([CA,TN]), false, false, [state#6]
   +- Scan OneRowRelation[]
```

**After**
```scala
scala> sql("select state from (select explode(array('CA','TN')) state) where state in ('TN','TN','TN','TN','TN','TN','TN')").explain
== Physical Plan ==
*Filter state#6 IN (TN)
+- Generate explode([CA,TN]), false, false, [state#6]
   +- Scan OneRowRelation[]
```

## How was this patch tested?

Pass the Jenkins tests (including a new testcase).

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13876 from dongjoon-hyun/SPARK-16174.
2016-07-07 19:45:43 +08:00
Reynold Xin 986b251401 [SPARK-16400][SQL] Remove InSet filter pushdown from Parquet
## What changes were proposed in this pull request?
This patch removes InSet filter pushdown from Parquet data source, since row-based pushdown is not beneficial to Spark and brings extra complexity to the code base.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #14076 from rxin/SPARK-16400.
2016-07-07 18:09:18 +08:00
gatorsmile 42279bff68 [SPARK-16374][SQL] Remove Alias from MetastoreRelation and SimpleCatalogRelation
#### What changes were proposed in this pull request?
Different from the other leaf nodes, `MetastoreRelation` and `SimpleCatalogRelation` have a pre-defined `alias`, which is used to change the qualifier of the node. However, based on the existing alias handling, alias should be put in `SubqueryAlias`.

This PR is to separate alias handling from `MetastoreRelation` and `SimpleCatalogRelation` to make it consistent with the other nodes. It simplifies the signature and conversion to a `BaseRelation`.

For example, below is an example query for `MetastoreRelation`,  which is converted to a `LogicalRelation`:
```SQL
SELECT tmp.a + 1 FROM test_parquet_ctas tmp WHERE tmp.a > 2
```

Before changes, the analyzed plan is
```
== Analyzed Logical Plan ==
(a + 1): int
Project [(a#951 + 1) AS (a + 1)#952]
+- Filter (a#951 > 2)
   +- SubqueryAlias tmp
      +- Relation[a#951] parquet
```
After changes, the analyzed plan becomes
```
== Analyzed Logical Plan ==
(a + 1): int
Project [(a#951 + 1) AS (a + 1)#952]
+- Filter (a#951 > 2)
   +- SubqueryAlias tmp
      +- SubqueryAlias test_parquet_ctas
         +- Relation[a#951] parquet
```

**Note: the optimized plans are the same.**

For `SimpleCatalogRelation`, the existing code always generates two Subqueries. Thus, no change is needed.

#### How was this patch tested?
Added test cases.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14053 from gatorsmile/removeAliasFromMetastoreRelation.
2016-07-07 12:07:19 +08:00
hyukjinkwon 34283de160 [SPARK-14839][SQL] Support for other types for tableProperty rule in SQL syntax
## What changes were proposed in this pull request?

Currently, Scala API supports to take options with the types, `String`, `Long`, `Double` and `Boolean` and Python API also supports other types.

This PR corrects `tableProperty` rule to support other types (string, boolean, double and integer) so that support the options for data sources in a consistent way. This will affect other rules such as DBPROPERTIES and TBLPROPERTIES (allowing other types as values).

Also, `TODO add bucketing and partitioning.` was removed because it was resolved in 24bea00047

## How was this patch tested?

Unit test in `MetastoreDataSourcesSuite.scala`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #13517 from HyukjinKwon/SPARK-14839.
2016-07-06 23:57:18 -04:00
Reynold Xin 8e3e4ed6c0 [SPARK-16371][SQL] Two follow-up tasks
## What changes were proposed in this pull request?
This is a small follow-up for SPARK-16371:

1. Hide removeMetadata from public API.
2. Add JIRA ticket number to test case name.

## How was this patch tested?
Updated a test comment.

Author: Reynold Xin <rxin@databricks.com>

Closes #14074 from rxin/parquet-filter.
2016-07-06 15:04:37 -07:00
Dongjoon Hyun d0d28507ca [SPARK-16286][SQL] Implement stack table generating function
## What changes were proposed in this pull request?

This PR implements `stack` table generating function.

## How was this patch tested?

Pass the Jenkins tests including new testcases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14033 from dongjoon-hyun/SPARK-16286.
2016-07-06 10:54:43 +08:00
Reynold Xin 16a2a7d714 [SPARK-16311][SQL] Metadata refresh should work on temporary views
## What changes were proposed in this pull request?
This patch fixes the bug that the refresh command does not work on temporary views. This patch is based on https://github.com/apache/spark/pull/13989, but removes the public Dataset.refresh() API as well as improved test coverage.

Note that I actually think the public refresh() API is very useful. We can in the future implement it by also invalidating the lazy vals in QueryExecution (or alternatively just create a new QueryExecution).

## How was this patch tested?
Re-enabled a previously ignored test, and added a new test suite for Hive testing behavior of temporary views against MetastoreRelation.

Author: Reynold Xin <rxin@databricks.com>
Author: petermaxlee <petermaxlee@gmail.com>

Closes #14009 from rxin/SPARK-16311.
2016-07-05 11:36:05 -07:00