Commit graph

6589 commits

Author SHA1 Message Date
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
Maxim Gekk 6a734575a8 [SPARK-23849][SQL] Tests for the samplingRatio option of JSON datasource
## What changes were proposed in this pull request?

Proposed tests checks that only subset of input dataset is touched during schema inferring.

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

Closes #20963 from MaxGekk/json-sampling-tests.
2018-04-07 21:44:32 -07:00
Huaxin Gao 2c1fe64757 [SPARK-23847][PYTHON][SQL] Add asc_nulls_first, asc_nulls_last to PySpark
## What changes were proposed in this pull request?

Column.scala and Functions.scala have asc_nulls_first, asc_nulls_last,  desc_nulls_first and desc_nulls_last. Add the corresponding python APIs in column.py and functions.py

## How was this patch tested?
Add doctest

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

Closes #20962 from huaxingao/spark-23847.
2018-04-08 12:09:06 +08: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
Li Jin d766ea2ff2 [SPARK-23861][SQL][DOC] Clarify default window frame with and without orderBy clause
## What changes were proposed in this pull request?

Add docstring to clarify default window frame boundaries with and without orderBy clause

## How was this patch tested?

Manually generate doc and check.

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

Closes #20978 from icexelloss/SPARK-23861-window-doc.
2018-04-07 00:15:54 +08:00
Yuchen Huo 9452401931 [SPARK-23822][SQL] Improve error message for Parquet schema mismatches
## What changes were proposed in this pull request?

This pull request tries to improve the error message for spark while reading parquet files with different schemas, e.g. One with a STRING column and the other with a INT column. A new ParquetSchemaColumnConvertNotSupportedException is added to replace the old UnsupportedOperationException. The Exception is again wrapped in FileScanRdd.scala to throw a more a general QueryExecutionException with the actual parquet file name which trigger the exception.

## How was this patch tested?

Unit tests added to check the new exception and verify the error messages.

Also manually tested with two parquet with different schema to check the error message.

<img width="1125" alt="screen shot 2018-03-30 at 4 03 04 pm" src="https://user-images.githubusercontent.com/37087310/38156580-dd58a140-3433-11e8-973a-b816d859fbe1.png">

Author: Yuchen Huo <yuchen.huo@databricks.com>

Closes #20953 from yuchenhuo/SPARK-23822.
2018-04-06 08:35:20 -07:00
Daniel Sakuma 6ade5cbb49 [MINOR][DOC] Fix some typos and grammar issues
## What changes were proposed in this pull request?

Easy fix in the documentation.

## How was this patch tested?

N/A

Closes #20948

Author: Daniel Sakuma <dsakuma@gmail.com>

Closes #20928 from dsakuma/fix_typo_configuration_docs.
2018-04-06 13:37:08 +08: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
Gengliang Wang d8379e5bc3 [SPARK-23838][WEBUI] Running SQL query is displayed as "completed" in SQL tab
## What changes were proposed in this pull request?

A running SQL query would appear as completed in the Spark UI:
![image1](https://user-images.githubusercontent.com/1097932/38170733-3d7cb00c-35bf-11e8-994c-43f2d4fa285d.png)

We can see the query in "Completed queries", while in in the job page we see it's still running Job 132.
![image2](https://user-images.githubusercontent.com/1097932/38170735-48f2c714-35bf-11e8-8a41-6fae23543c46.png)

After some time in the query still appears in "Completed queries" (while it's still running), but the "Duration" gets increased.
![image3](https://user-images.githubusercontent.com/1097932/38170737-50f87ea4-35bf-11e8-8b60-000f6f918964.png)

To reproduce, we can run a query with multiple jobs. E.g. Run TPCDS q6.

The reason is that updates from executions are written into kvstore periodically, and the job start event may be missed.

## How was this patch tested?
Manually run the job again and check the SQL Tab. The fix is pretty simple.

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

Closes #20955 from gengliangwang/jobCompleted.
2018-04-04 15:43:58 -07: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
gatorsmile 16ef6baa36 [SPARK-23826][TEST] TestHiveSparkSession should set default session
## What changes were proposed in this pull request?
In TestHive, the base spark session does this in getOrCreate(), we emulate that behavior for tests.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20969 from gatorsmile/setDefault.
2018-04-04 14:31:03 +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
Eric Liang 359375eff7 [SPARK-23809][SQL] Active SparkSession should be set by getOrCreate
## What changes were proposed in this pull request?

Currently, the active spark session is set inconsistently (e.g., in createDataFrame, prior to query execution). Many places in spark also incorrectly query active session when they should be calling activeSession.getOrElse(defaultSession) and so might get None even if a Spark session exists.

The semantics here can be cleaned up if we also set the active session when the default session is set.

Related: https://github.com/apache/spark/pull/20926/files

## How was this patch tested?

Unit test, existing test. Note that if https://github.com/apache/spark/pull/20926 merges first we should also update the tests there.

Author: Eric Liang <ekl@databricks.com>

Closes #20927 from ericl/active-session-cleanup.
2018-04-03 17:09:12 -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
Jose Torres 66a3a5a2dc [SPARK-23099][SS] Migrate foreach sink to DataSourceV2
## What changes were proposed in this pull request?

Migrate foreach sink to DataSourceV2.

Since the previous attempt at this PR #20552, we've changed and strictly defined the lifecycle of writer components. This means we no longer need the complicated lifecycle shim from that PR; it just naturally works.

## How was this patch tested?

existing tests

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

Closes #20951 from jose-torres/foreach.
2018-04-03 11:05:29 -07: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
Tathagata Das 15298b99ac [SPARK-23827][SS] StreamingJoinExec should ensure that input data is partitioned into specific number of partitions
## What changes were proposed in this pull request?

Currently, the requiredChildDistribution does not specify the partitions. This can cause the weird corner cases where the child's distribution is `SinglePartition` which satisfies the required distribution of `ClusterDistribution(no-num-partition-requirement)`, thus eliminating the shuffle needed to repartition input data into the required number of partitions (i.e. same as state stores). That can lead to "file not found" errors on the state store delta files as the micro-batch-with-no-shuffle will not run certain tasks and therefore not generate the expected state store delta files.

This PR adds the required constraint on the number of partitions.

## How was this patch tested?
Modified test harness to always check that ANY stateful operator should have a constraint on the number of partitions. As part of that, the existing opt-in checks on child output partitioning were removed, as they are redundant.

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

Closes #20941 from tdas/SPARK-23827.
2018-03-30 16:48:26 -07: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
Jose Torres 5b5a36ed6d Roll forward "[SPARK-23096][SS] Migrate rate source to V2"
## What changes were proposed in this pull request?

Roll forward c68ec4e (#20688).

There are two minor test changes required:

* An error which used to be TreeNodeException[ArithmeticException] is no longer wrapped and is now just ArithmeticException.
* The test framework simply does not set the active Spark session. (Or rather, it doesn't do so early enough - I think it only happens when a query is analyzed.) I've added the required logic to SQLTestUtils.

## How was this patch tested?

existing tests

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

Closes #20922 from jose-torres/ratefix.
2018-03-30 21:54:26 +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
Jongyoul Lee df05fb63ab [SPARK-23743][SQL] Changed a comparison logic from containing 'slf4j' to starting with 'org.slf4j'
## What changes were proposed in this pull request?
isSharedClass returns if some classes can/should be shared or not. It checks if the classes names have some keywords or start with some names. Following the logic, it can occur unintended behaviors when a custom package has `slf4j` inside the package or class name. As I guess, the first intention seems to figure out the class containing `org.slf4j`. It would be better to change the comparison logic to `name.startsWith("org.slf4j")`

## How was this patch tested?
This patch should pass all of the current tests and keep all of the current behaviors. In my case, I'm using ProtobufDeserializer to get a table schema from hive tables. Thus some Protobuf packages and names have `slf4j` inside. Without this patch, it cannot be resolved because of ClassCastException from different classloaders.

Author: Jongyoul Lee <jongyoul@gmail.com>

Closes #20860 from jongyoul/SPARK-23743.
2018-03-30 14:07:35 +08:00
Jose Torres b348901192 [SPARK-23808][SQL] Set default Spark session in test-only spark sessions.
## What changes were proposed in this pull request?

Set default Spark session in the TestSparkSession and TestHiveSparkSession constructors.

## How was this patch tested?

new unit tests

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

Closes #20926 from jose-torres/test3.
2018-03-29 21:36:56 -07:00
Kent Yao a7755fd8ce [SPARK-23639][SQL] Obtain token before init metastore client in SparkSQL CLI
## What changes were proposed in this pull request?

In SparkSQLCLI, SessionState generates before SparkContext instantiating. When we use --proxy-user to impersonate, it's unable to initializing a metastore client to talk to the secured metastore for no kerberos ticket.

This PR use real user ugi to obtain token for owner before talking to kerberized metastore.

## How was this patch tested?

Manually verified with kerberized hive metasotre / hdfs.

Author: Kent Yao <yaooqinn@hotmail.com>

Closes #20784 from yaooqinn/SPARK-23639.
2018-03-29 10:46:28 -07:00
gatorsmile 761565a3cc Revert "[SPARK-23096][SS] Migrate rate source to V2"
This reverts commit c68ec4e6a1.
2018-03-28 09:11:52 -07: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
jerryshao c68ec4e6a1 [SPARK-23096][SS] Migrate rate source to V2
## What changes were proposed in this pull request?

This PR migrate micro batch rate source to V2 API and rewrite UTs to suite V2 test.

## How was this patch tested?

UTs.

Author: jerryshao <sshao@hortonworks.com>

Closes #20688 from jerryshao/SPARK-23096.
2018-03-27 14:39:05 -07: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
Takeshi Yamamuro 5f653d4f7c [SPARK-23167][SQL] Add TPCDS queries v2.7 in TPCDSQuerySuite
## What changes were proposed in this pull request?
This pr added TPCDS v2.7 (latest) queries in `TPCDSQuerySuite` because the current `TPCDSQuerySuite` tests older one (v1.4) and some queries are different from v1.4 and v2.7. Since the original v2.7 queries have the syntaxes that Spark cannot parse, I changed these queries in a following way:

 - [date] + 14 days -> date + `INTERVAL` 14 days
 - [column name] as "30 days" -> [column name] as \`30 days\`
 - Fix some syntax errors, e.g., missing brackets

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20343 from maropu/TPCDSV2_7.
2018-03-25 09:18:26 -07:00
Jose Torres 816a5496ba [SPARK-23788][SS] Fix race in StreamingQuerySuite
## What changes were proposed in this pull request?

The serializability test uses the same MemoryStream instance for 3 different queries. If any of those queries ask it to commit before the others have run, the rest will see empty dataframes. This can fail the test if q3 is affected.

We should use one instance per query instead.

## How was this patch tested?

Existing unit test. If I move q2.processAllAvailable() before starting q3, the test always fails without the fix.

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

Closes #20896 from jose-torres/fixrace.
2018-03-24 18:21:01 -07:00
Liang-Chi Hsieh b2edc30db1 [SPARK-23614][SQL] Fix incorrect reuse exchange when caching is used
## What changes were proposed in this pull request?

We should provide customized canonicalize plan for `InMemoryRelation` and `InMemoryTableScanExec`. Otherwise, we can wrongly treat two different cached plans as same result. It causes wrongly reused exchange then.

For a test query like this:
```scala
val cached = spark.createDataset(Seq(TestDataUnion(1, 2, 3), TestDataUnion(4, 5, 6))).cache()
val group1 = cached.groupBy("x").agg(min(col("y")) as "value")
val group2 = cached.groupBy("x").agg(min(col("z")) as "value")
group1.union(group2)
```

Canonicalized plans before:

First exchange:
```
Exchange hashpartitioning(none#0, 5)
+- *(1) HashAggregate(keys=[none#0], functions=[partial_min(none#1)], output=[none#0, none#4])
   +- *(1) InMemoryTableScan [none#0, none#1]
         +- InMemoryRelation [x#4253, y#4254, z#4255], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)
               +- LocalTableScan [x#4253, y#4254, z#4255]
```

Second exchange:
```
Exchange hashpartitioning(none#0, 5)
+- *(3) HashAggregate(keys=[none#0], functions=[partial_min(none#1)], output=[none#0, none#4])
   +- *(3) InMemoryTableScan [none#0, none#1]
         +- InMemoryRelation [x#4253, y#4254, z#4255], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)
               +- LocalTableScan [x#4253, y#4254, z#4255]
```

You can find that they have the canonicalized plans are the same, although we use different columns in two `InMemoryTableScan`s.

Canonicalized plan after:

First exchange:
```
Exchange hashpartitioning(none#0, 5)
+- *(1) HashAggregate(keys=[none#0], functions=[partial_min(none#1)], output=[none#0, none#4])
   +- *(1) InMemoryTableScan [none#0, none#1]
         +- InMemoryRelation [none#0, none#1, none#2], true, 10000, StorageLevel(memory, 1 replicas)
               +- LocalTableScan [none#0, none#1, none#2]
```

Second exchange:
```
Exchange hashpartitioning(none#0, 5)
+- *(3) HashAggregate(keys=[none#0], functions=[partial_min(none#1)], output=[none#0, none#4])
   +- *(3) InMemoryTableScan [none#0, none#2]
         +- InMemoryRelation [none#0, none#1, none#2], true, 10000, StorageLevel(memory, 1 replicas)
               +- LocalTableScan [none#0, none#1, none#2]
```

## How was this patch tested?

Added unit test.

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

Closes #20831 from viirya/SPARK-23614.
2018-03-22 21:23:25 -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
Dilip Biswal 5c9eaa6b58 [SPARK-23372][SQL] Writing empty struct in parquet fails during execution. It should fail earlier in the processing.
## What changes were proposed in this pull request?
Currently we allow writing data frames with empty schema into a file based datasource for certain file formats such as JSON, ORC etc. For formats such as Parquet and Text, we raise error at different times of execution. For text format, we return error from the driver early on in processing where as for format such as parquet, the error is raised from executor.

**Example**
spark.emptyDataFrame.write.format("parquet").mode("overwrite").save(path)
**Results in**
``` SQL
org.apache.parquet.schema.InvalidSchemaException: Cannot write a schema with an empty group: message spark_schema {
 }

at org.apache.parquet.schema.TypeUtil$1.visit(TypeUtil.java:27)
 at org.apache.parquet.schema.TypeUtil$1.visit(TypeUtil.java:37)
 at org.apache.parquet.schema.MessageType.accept(MessageType.java:58)
 at org.apache.parquet.schema.TypeUtil.checkValidWriteSchema(TypeUtil.java:23)
 at org.apache.parquet.hadoop.ParquetFileWriter.<init>(ParquetFileWriter.java:225)
 at org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:342)
 at org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:302)
 at org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.<init>(ParquetOutputWriter.scala:37)
 at org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$$anon$1.newInstance(ParquetFileFormat.scala:151)
 at org.apache.spark.sql.execution.datasources.FileFormatWriter$SingleDirectoryWriteTask.newOutputWriter(FileFormatWriter.scala:376)
 at org.apache.spark.sql.execution.datasources.FileFormatWriter$SingleDirectoryWriteTask.execute(FileFormatWriter.scala:387)
 at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask$3.apply(FileFormatWriter.scala:278)
 at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask$3.apply(FileFormatWriter.scala:276)
 at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1411)
 at org.apache.spark.sql.execution.datasources.FileFormatWriter$.org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask(FileFormatWriter.scala:281)
 at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:206)
 at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:205)
 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.
```

In this PR, we unify the error processing and raise error on attempt to write empty schema based dataframes into file based datasource (orc, parquet, text , csv, json etc) early on in the processing.

## How was this patch tested?

Unit tests added in FileBasedDatasourceSuite.

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

Closes #20579 from dilipbiswal/spark-23372.
2018-03-21 21:49:02 -07: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
Gabor Somogyi 918c7e99af [SPARK-23288][SS] Fix output metrics with parquet sink
## What changes were proposed in this pull request?

Output metrics were not filled when parquet sink used.

This PR fixes this problem by passing a `BasicWriteJobStatsTracker` in `FileStreamSink`.

## How was this patch tested?

Additional unit test added.

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

Closes #20745 from gaborgsomogyi/SPARK-23288.
2018-03-21 10:06:26 -07:00
Takeshi Yamamuro 98d0ea3f60 [SPARK-23264][SQL] Fix scala.MatchError in literals.sql.out
## What changes were proposed in this pull request?
To fix `scala.MatchError` in `literals.sql.out`, this pr added an entry for `CalendarIntervalType` in `QueryExecution.toHiveStructString`.

## How was this patch tested?
Existing tests and added tests in `literals.sql`

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20872 from maropu/FixIntervalTests.
2018-03-21 09:52:28 -07:00
hyukjinkwon 8d79113b81 [SPARK-23577][SQL] Supports custom line separator for text 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.

## How was this patch tested?

Manual tests and unit tests were added.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20727 from HyukjinKwon/linesep-text.
2018-03-21 09:46:47 -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
Jose Torres 2c4b9962fd [SPARK-23574][SQL] Report SinglePartition in DataSourceV2ScanExec when there's exactly 1 data reader factory.
## What changes were proposed in this pull request?

Report SinglePartition in DataSourceV2ScanExec when there's exactly 1 data reader factory.

Note that this means reader factories end up being constructed as partitioning is checked; let me know if you think that could be a problem.

## How was this patch tested?

existing unit tests

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

Closes #20726 from jose-torres/SPARK-23574.
2018-03-20 11:46:51 -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
Dongjoon Hyun 5414abca4f [SPARK-23553][TESTS] Tests should not assume the default value of spark.sql.sources.default
## What changes were proposed in this pull request?

Currently, some tests have an assumption that `spark.sql.sources.default=parquet`. In fact, that is a correct assumption, but that assumption makes it difficult to test new data source format.

This PR aims to
- Improve test suites more robust and makes it easy to test new data sources in the future.
- Test new native ORC data source with the full existing Apache Spark test coverage.

As an example, the PR uses `spark.sql.sources.default=orc` during reviews. The value should be `parquet` when this PR is accepted.

## How was this patch tested?

Pass the Jenkins with updated tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20705 from dongjoon-hyun/SPARK-23553.
2018-03-16 09:36:30 -07:00
myroslavlisniak c2632edebd [SPARK-23670][SQL] Fix memory leak on SparkPlanGraphWrapper
Clean up SparkPlanGraphWrapper objects from InMemoryStore together with cleaning up SQLExecutionUIData
existing unit test was extended to check also SparkPlanGraphWrapper object count

vanzin

Author: myroslavlisniak <acnipin@gmail.com>

Closes #20813 from myroslavlisniak/master.
2018-03-15 17:20:59 -07:00
Yuming Wang 15c3c98300 [HOT-FIX] Fix SparkOutOfMemoryError: Unable to acquire 262144 bytes of memory, got 224631
## What changes were proposed in this pull request?

https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/88263/testReport
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/88260/testReport
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/88257/testReport
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/88224/testReport

These tests all failed:
```
org.apache.spark.memory.SparkOutOfMemoryError:  Unable to acquire 262144 bytes of memory, got 224631
at org.apache.spark.memory.MemoryConsumer.throwOom(MemoryConsumer.java:157)
at org.apache.spark.memory.MemoryConsumer.allocateArray(MemoryConsumer.java:98)
at org.apache.spark.unsafe.map.BytesToBytesMap.allocate(BytesToBytesMap.java:787)
at org.apache.spark.unsafe.map.BytesToBytesMap.<init>(BytesToBytesMap.java:204)
at org.apache.spark.unsafe.map.BytesToBytesMap.<init>(BytesToBytesMap.java:219)
...
```

This PR ignore this test.

## How was this patch tested?

N/A

Author: Yuming Wang <yumwang@ebay.com>

Closes #20835 from wangyum/SPARK-23598.
2018-03-15 19:54:58 +01:00
Yuanjian Li 7c3e8995f1 [SPARK-23533][SS] Add support for changing ContinuousDataReader's startOffset
## What changes were proposed in this pull request?

As discussion in #20675, we need add a new interface `ContinuousDataReaderFactory` to support the requirements of setting start offset in Continuous Processing.

## How was this patch tested?

Existing UT.

Author: Yuanjian Li <xyliyuanjian@gmail.com>

Closes #20689 from xuanyuanking/SPARK-23533.
2018-03-15 00:04:28 -07:00
Kazuaki Ishizaki 1098933b0a [SPARK-23598][SQL] Make methods in BufferedRowIterator public to avoid runtime error for a large query
## What changes were proposed in this pull request?

This PR fixes runtime error regarding a large query when a generated code has split classes. The issue is `append()`, `stopEarly()`, and other methods are not accessible from split classes that are not subclasses of `BufferedRowIterator`.
This PR fixes this issue by making them `public`.

Before applying the PR, we see the following exception by running the attached program with `CodeGenerator.GENERATED_CLASS_SIZE_THRESHOLD=-1`.
```
  test("SPARK-23598") {
    // When set -1 to CodeGenerator.GENERATED_CLASS_SIZE_THRESHOLD, an exception is thrown
    val df_pet_age = Seq((8, "bat"), (15, "mouse"), (5, "horse")).toDF("age", "name")
    df_pet_age.groupBy("name").avg("age").show()
  }
```

Exception:
```
19:40:52.591 WARN org.apache.hadoop.util.NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
19:41:32.319 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 0.0 (TID 0)
java.lang.IllegalAccessError: tried to access method org.apache.spark.sql.execution.BufferedRowIterator.shouldStop()Z from class org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1$agg_NestedClass1
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1$agg_NestedClass1.agg_doAggregateWithKeys$(generated.java:203)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(generated.java:160)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$11$$anon$1.hasNext(WholeStageCodegenExec.scala:616)
	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
	at org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:125)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)
	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:745)
...
```

Generated code (line 195 calles `stopEarly()`).
```
/* 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;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private boolean agg_initAgg;
/* 010 */   private boolean agg_bufIsNull;
/* 011 */   private double agg_bufValue;
/* 012 */   private boolean agg_bufIsNull1;
/* 013 */   private long agg_bufValue1;
/* 014 */   private agg_FastHashMap agg_fastHashMap;
/* 015 */   private org.apache.spark.unsafe.KVIterator<UnsafeRow, UnsafeRow> agg_fastHashMapIter;
/* 016 */   private org.apache.spark.unsafe.KVIterator agg_mapIter;
/* 017 */   private org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap agg_hashMap;
/* 018 */   private org.apache.spark.sql.execution.UnsafeKVExternalSorter agg_sorter;
/* 019 */   private scala.collection.Iterator inputadapter_input;
/* 020 */   private boolean agg_agg_isNull11;
/* 021 */   private boolean agg_agg_isNull25;
/* 022 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder[] agg_mutableStateArray1 = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder[2];
/* 023 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] agg_mutableStateArray2 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[2];
/* 024 */   private UnsafeRow[] agg_mutableStateArray = new UnsafeRow[2];
/* 025 */
/* 026 */   public GeneratedIteratorForCodegenStage1(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 */
/* 034 */     agg_fastHashMap = new agg_FastHashMap(((org.apache.spark.sql.execution.aggregate.HashAggregateExec) references[0] /* plan */).getTaskMemoryManager(), ((org.apache.spark.sql.execution.aggregate.HashAggregateExec) references[0] /* plan */).getEmptyAggregationBuffer());
/* 035 */     agg_hashMap = ((org.apache.spark.sql.execution.aggregate.HashAggregateExec) references[0] /* plan */).createHashMap();
/* 036 */     inputadapter_input = inputs[0];
/* 037 */     agg_mutableStateArray[0] = new UnsafeRow(1);
/* 038 */     agg_mutableStateArray1[0] = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(agg_mutableStateArray[0], 32);
/* 039 */     agg_mutableStateArray2[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(agg_mutableStateArray1[0], 1);
/* 040 */     agg_mutableStateArray[1] = new UnsafeRow(3);
/* 041 */     agg_mutableStateArray1[1] = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(agg_mutableStateArray[1], 32);
/* 042 */     agg_mutableStateArray2[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(agg_mutableStateArray1[1], 3);
/* 043 */
/* 044 */   }
/* 045 */
/* 046 */   public class agg_FastHashMap {
/* 047 */     private org.apache.spark.sql.catalyst.expressions.RowBasedKeyValueBatch batch;
/* 048 */     private int[] buckets;
/* 049 */     private int capacity = 1 << 16;
/* 050 */     private double loadFactor = 0.5;
/* 051 */     private int numBuckets = (int) (capacity / loadFactor);
/* 052 */     private int maxSteps = 2;
/* 053 */     private int numRows = 0;
/* 054 */     private org.apache.spark.sql.types.StructType keySchema = new org.apache.spark.sql.types.StructType().add(((java.lang.String) references[1] /* keyName */), org.apache.spark.sql.types.DataTypes.StringType);
/* 055 */     private org.apache.spark.sql.types.StructType valueSchema = new org.apache.spark.sql.types.StructType().add(((java.lang.String) references[2] /* keyName */), org.apache.spark.sql.types.DataTypes.DoubleType)
/* 056 */     .add(((java.lang.String) references[3] /* keyName */), org.apache.spark.sql.types.DataTypes.LongType);
/* 057 */     private Object emptyVBase;
/* 058 */     private long emptyVOff;
/* 059 */     private int emptyVLen;
/* 060 */     private boolean isBatchFull = false;
/* 061 */
/* 062 */     public agg_FastHashMap(
/* 063 */       org.apache.spark.memory.TaskMemoryManager taskMemoryManager,
/* 064 */       InternalRow emptyAggregationBuffer) {
/* 065 */       batch = org.apache.spark.sql.catalyst.expressions.RowBasedKeyValueBatch
/* 066 */       .allocate(keySchema, valueSchema, taskMemoryManager, capacity);
/* 067 */
/* 068 */       final UnsafeProjection valueProjection = UnsafeProjection.create(valueSchema);
/* 069 */       final byte[] emptyBuffer = valueProjection.apply(emptyAggregationBuffer).getBytes();
/* 070 */
/* 071 */       emptyVBase = emptyBuffer;
/* 072 */       emptyVOff = Platform.BYTE_ARRAY_OFFSET;
/* 073 */       emptyVLen = emptyBuffer.length;
/* 074 */
/* 075 */       buckets = new int[numBuckets];
/* 076 */       java.util.Arrays.fill(buckets, -1);
/* 077 */     }
/* 078 */
/* 079 */     public org.apache.spark.sql.catalyst.expressions.UnsafeRow findOrInsert(UTF8String agg_key) {
/* 080 */       long h = hash(agg_key);
/* 081 */       int step = 0;
/* 082 */       int idx = (int) h & (numBuckets - 1);
/* 083 */       while (step < maxSteps) {
/* 084 */         // Return bucket index if it's either an empty slot or already contains the key
/* 085 */         if (buckets[idx] == -1) {
/* 086 */           if (numRows < capacity && !isBatchFull) {
/* 087 */             // creating the unsafe for new entry
/* 088 */             UnsafeRow agg_result = new UnsafeRow(1);
/* 089 */             org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder
/* 090 */             = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(agg_result,
/* 091 */               32);
/* 092 */             org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter agg_rowWriter
/* 093 */             = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(
/* 094 */               agg_holder,
/* 095 */               1);
/* 096 */             agg_holder.reset(); //TODO: investigate if reset or zeroout are actually needed
/* 097 */             agg_rowWriter.zeroOutNullBytes();
/* 098 */             agg_rowWriter.write(0, agg_key);
/* 099 */             agg_result.setTotalSize(agg_holder.totalSize());
/* 100 */             Object kbase = agg_result.getBaseObject();
/* 101 */             long koff = agg_result.getBaseOffset();
/* 102 */             int klen = agg_result.getSizeInBytes();
/* 103 */
/* 104 */             UnsafeRow vRow
/* 105 */             = batch.appendRow(kbase, koff, klen, emptyVBase, emptyVOff, emptyVLen);
/* 106 */             if (vRow == null) {
/* 107 */               isBatchFull = true;
/* 108 */             } else {
/* 109 */               buckets[idx] = numRows++;
/* 110 */             }
/* 111 */             return vRow;
/* 112 */           } else {
/* 113 */             // No more space
/* 114 */             return null;
/* 115 */           }
/* 116 */         } else if (equals(idx, agg_key)) {
/* 117 */           return batch.getValueRow(buckets[idx]);
/* 118 */         }
/* 119 */         idx = (idx + 1) & (numBuckets - 1);
/* 120 */         step++;
/* 121 */       }
/* 122 */       // Didn't find it
/* 123 */       return null;
/* 124 */     }
/* 125 */
/* 126 */     private boolean equals(int idx, UTF8String agg_key) {
/* 127 */       UnsafeRow row = batch.getKeyRow(buckets[idx]);
/* 128 */       return (row.getUTF8String(0).equals(agg_key));
/* 129 */     }
/* 130 */
/* 131 */     private long hash(UTF8String agg_key) {
/* 132 */       long agg_hash = 0;
/* 133 */
/* 134 */       int agg_result = 0;
/* 135 */       byte[] agg_bytes = agg_key.getBytes();
/* 136 */       for (int i = 0; i < agg_bytes.length; i++) {
/* 137 */         int agg_hash1 = agg_bytes[i];
/* 138 */         agg_result = (agg_result ^ (0x9e3779b9)) + agg_hash1 + (agg_result << 6) + (agg_result >>> 2);
/* 139 */       }
/* 140 */
/* 141 */       agg_hash = (agg_hash ^ (0x9e3779b9)) + agg_result + (agg_hash << 6) + (agg_hash >>> 2);
/* 142 */
/* 143 */       return agg_hash;
/* 144 */     }
/* 145 */
/* 146 */     public org.apache.spark.unsafe.KVIterator<UnsafeRow, UnsafeRow> rowIterator() {
/* 147 */       return batch.rowIterator();
/* 148 */     }
/* 149 */
/* 150 */     public void close() {
/* 151 */       batch.close();
/* 152 */     }
/* 153 */
/* 154 */   }
/* 155 */
/* 156 */   protected void processNext() throws java.io.IOException {
/* 157 */     if (!agg_initAgg) {
/* 158 */       agg_initAgg = true;
/* 159 */       long wholestagecodegen_beforeAgg = System.nanoTime();
/* 160 */       agg_nestedClassInstance1.agg_doAggregateWithKeys();
/* 161 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[8] /* aggTime */).add((System.nanoTime() - wholestagecodegen_beforeAgg) / 1000000);
/* 162 */     }
/* 163 */
/* 164 */     // output the result
/* 165 */
/* 166 */     while (agg_fastHashMapIter.next()) {
/* 167 */       UnsafeRow agg_aggKey = (UnsafeRow) agg_fastHashMapIter.getKey();
/* 168 */       UnsafeRow agg_aggBuffer = (UnsafeRow) agg_fastHashMapIter.getValue();
/* 169 */       wholestagecodegen_nestedClassInstance.agg_doAggregateWithKeysOutput(agg_aggKey, agg_aggBuffer);
/* 170 */
/* 171 */       if (shouldStop()) return;
/* 172 */     }
/* 173 */     agg_fastHashMap.close();
/* 174 */
/* 175 */     while (agg_mapIter.next()) {
/* 176 */       UnsafeRow agg_aggKey = (UnsafeRow) agg_mapIter.getKey();
/* 177 */       UnsafeRow agg_aggBuffer = (UnsafeRow) agg_mapIter.getValue();
/* 178 */       wholestagecodegen_nestedClassInstance.agg_doAggregateWithKeysOutput(agg_aggKey, agg_aggBuffer);
/* 179 */
/* 180 */       if (shouldStop()) return;
/* 181 */     }
/* 182 */
/* 183 */     agg_mapIter.close();
/* 184 */     if (agg_sorter == null) {
/* 185 */       agg_hashMap.free();
/* 186 */     }
/* 187 */   }
/* 188 */
/* 189 */   private wholestagecodegen_NestedClass wholestagecodegen_nestedClassInstance = new wholestagecodegen_NestedClass();
/* 190 */   private agg_NestedClass1 agg_nestedClassInstance1 = new agg_NestedClass1();
/* 191 */   private agg_NestedClass agg_nestedClassInstance = new agg_NestedClass();
/* 192 */
/* 193 */   private class agg_NestedClass1 {
/* 194 */     private void agg_doAggregateWithKeys() throws java.io.IOException {
/* 195 */       while (inputadapter_input.hasNext() && !stopEarly()) {
/* 196 */         InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 197 */         int inputadapter_value = inputadapter_row.getInt(0);
/* 198 */         boolean inputadapter_isNull1 = inputadapter_row.isNullAt(1);
/* 199 */         UTF8String inputadapter_value1 = inputadapter_isNull1 ?
/* 200 */         null : (inputadapter_row.getUTF8String(1));
/* 201 */
/* 202 */         agg_nestedClassInstance.agg_doConsume(inputadapter_row, inputadapter_value, inputadapter_value1, inputadapter_isNull1);
/* 203 */         if (shouldStop()) return;
/* 204 */       }
/* 205 */
/* 206 */       agg_fastHashMapIter = agg_fastHashMap.rowIterator();
/* 207 */       agg_mapIter = ((org.apache.spark.sql.execution.aggregate.HashAggregateExec) references[0] /* plan */).finishAggregate(agg_hashMap, agg_sorter, ((org.apache.spark.sql.execution.metric.SQLMetric) references[4] /* peakMemory */), ((org.apache.spark.sql.execution.metric.SQLMetric) references[5] /* spillSize */), ((org.apache.spark.sql.execution.metric.SQLMetric) references[6] /* avgHashProbe */));
/* 208 */
/* 209 */     }
/* 210 */
/* 211 */   }
/* 212 */
/* 213 */   private class wholestagecodegen_NestedClass {
/* 214 */     private void agg_doAggregateWithKeysOutput(UnsafeRow agg_keyTerm, UnsafeRow agg_bufferTerm)
/* 215 */     throws java.io.IOException {
/* 216 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[7] /* numOutputRows */).add(1);
/* 217 */
/* 218 */       boolean agg_isNull35 = agg_keyTerm.isNullAt(0);
/* 219 */       UTF8String agg_value37 = agg_isNull35 ?
/* 220 */       null : (agg_keyTerm.getUTF8String(0));
/* 221 */       boolean agg_isNull36 = agg_bufferTerm.isNullAt(0);
/* 222 */       double agg_value38 = agg_isNull36 ?
/* 223 */       -1.0 : (agg_bufferTerm.getDouble(0));
/* 224 */       boolean agg_isNull37 = agg_bufferTerm.isNullAt(1);
/* 225 */       long agg_value39 = agg_isNull37 ?
/* 226 */       -1L : (agg_bufferTerm.getLong(1));
/* 227 */
/* 228 */       agg_mutableStateArray1[1].reset();
/* 229 */
/* 230 */       agg_mutableStateArray2[1].zeroOutNullBytes();
/* 231 */
/* 232 */       if (agg_isNull35) {
/* 233 */         agg_mutableStateArray2[1].setNullAt(0);
/* 234 */       } else {
/* 235 */         agg_mutableStateArray2[1].write(0, agg_value37);
/* 236 */       }
/* 237 */
/* 238 */       if (agg_isNull36) {
/* 239 */         agg_mutableStateArray2[1].setNullAt(1);
/* 240 */       } else {
/* 241 */         agg_mutableStateArray2[1].write(1, agg_value38);
/* 242 */       }
/* 243 */
/* 244 */       if (agg_isNull37) {
/* 245 */         agg_mutableStateArray2[1].setNullAt(2);
/* 246 */       } else {
/* 247 */         agg_mutableStateArray2[1].write(2, agg_value39);
/* 248 */       }
/* 249 */       agg_mutableStateArray[1].setTotalSize(agg_mutableStateArray1[1].totalSize());
/* 250 */       append(agg_mutableStateArray[1]);
/* 251 */
/* 252 */     }
/* 253 */
/* 254 */   }
/* 255 */
/* 256 */   private class agg_NestedClass {
/* 257 */     private void agg_doConsume(InternalRow inputadapter_row, int agg_expr_0, UTF8String agg_expr_1, boolean agg_exprIsNull_1) throws java.io.IOException {
/* 258 */       UnsafeRow agg_unsafeRowAggBuffer = null;
/* 259 */       UnsafeRow agg_fastAggBuffer = null;
/* 260 */
/* 261 */       if (true) {
/* 262 */         if (!agg_exprIsNull_1) {
/* 263 */           agg_fastAggBuffer = agg_fastHashMap.findOrInsert(
/* 264 */             agg_expr_1);
/* 265 */         }
/* 266 */       }
/* 267 */       // Cannot find the key in fast hash map, try regular hash map.
/* 268 */       if (agg_fastAggBuffer == null) {
/* 269 */         // generate grouping key
/* 270 */         agg_mutableStateArray1[0].reset();
/* 271 */
/* 272 */         agg_mutableStateArray2[0].zeroOutNullBytes();
/* 273 */
/* 274 */         if (agg_exprIsNull_1) {
/* 275 */           agg_mutableStateArray2[0].setNullAt(0);
/* 276 */         } else {
/* 277 */           agg_mutableStateArray2[0].write(0, agg_expr_1);
/* 278 */         }
/* 279 */         agg_mutableStateArray[0].setTotalSize(agg_mutableStateArray1[0].totalSize());
/* 280 */         int agg_value7 = 42;
/* 281 */
/* 282 */         if (!agg_exprIsNull_1) {
/* 283 */           agg_value7 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(agg_expr_1.getBaseObject(), agg_expr_1.getBaseOffset(), agg_expr_1.numBytes(), agg_value7);
/* 284 */         }
/* 285 */         if (true) {
/* 286 */           // try to get the buffer from hash map
/* 287 */           agg_unsafeRowAggBuffer =
/* 288 */           agg_hashMap.getAggregationBufferFromUnsafeRow(agg_mutableStateArray[0], agg_value7);
/* 289 */         }
/* 290 */         // Can't allocate buffer from the hash map. Spill the map and fallback to sort-based
/* 291 */         // aggregation after processing all input rows.
/* 292 */         if (agg_unsafeRowAggBuffer == null) {
/* 293 */           if (agg_sorter == null) {
/* 294 */             agg_sorter = agg_hashMap.destructAndCreateExternalSorter();
/* 295 */           } else {
/* 296 */             agg_sorter.merge(agg_hashMap.destructAndCreateExternalSorter());
/* 297 */           }
/* 298 */
/* 299 */           // the hash map had be spilled, it should have enough memory now,
/* 300 */           // try to allocate buffer again.
/* 301 */           agg_unsafeRowAggBuffer = agg_hashMap.getAggregationBufferFromUnsafeRow(
/* 302 */             agg_mutableStateArray[0], agg_value7);
/* 303 */           if (agg_unsafeRowAggBuffer == null) {
/* 304 */             // failed to allocate the first page
/* 305 */             throw new OutOfMemoryError("No enough memory for aggregation");
/* 306 */           }
/* 307 */         }
/* 308 */
/* 309 */       }
/* 310 */
/* 311 */       if (agg_fastAggBuffer != null) {
/* 312 */         // common sub-expressions
/* 313 */         boolean agg_isNull21 = false;
/* 314 */         long agg_value23 = -1L;
/* 315 */         if (!false) {
/* 316 */           agg_value23 = (long) agg_expr_0;
/* 317 */         }
/* 318 */         // evaluate aggregate function
/* 319 */         boolean agg_isNull23 = true;
/* 320 */         double agg_value25 = -1.0;
/* 321 */
/* 322 */         boolean agg_isNull24 = agg_fastAggBuffer.isNullAt(0);
/* 323 */         double agg_value26 = agg_isNull24 ?
/* 324 */         -1.0 : (agg_fastAggBuffer.getDouble(0));
/* 325 */         if (!agg_isNull24) {
/* 326 */           agg_agg_isNull25 = true;
/* 327 */           double agg_value27 = -1.0;
/* 328 */           do {
/* 329 */             boolean agg_isNull26 = agg_isNull21;
/* 330 */             double agg_value28 = -1.0;
/* 331 */             if (!agg_isNull21) {
/* 332 */               agg_value28 = (double) agg_value23;
/* 333 */             }
/* 334 */             if (!agg_isNull26) {
/* 335 */               agg_agg_isNull25 = false;
/* 336 */               agg_value27 = agg_value28;
/* 337 */               continue;
/* 338 */             }
/* 339 */
/* 340 */             boolean agg_isNull27 = false;
/* 341 */             double agg_value29 = -1.0;
/* 342 */             if (!false) {
/* 343 */               agg_value29 = (double) 0;
/* 344 */             }
/* 345 */             if (!agg_isNull27) {
/* 346 */               agg_agg_isNull25 = false;
/* 347 */               agg_value27 = agg_value29;
/* 348 */               continue;
/* 349 */             }
/* 350 */
/* 351 */           } while (false);
/* 352 */
/* 353 */           agg_isNull23 = false; // resultCode could change nullability.
/* 354 */           agg_value25 = agg_value26 + agg_value27;
/* 355 */
/* 356 */         }
/* 357 */         boolean agg_isNull29 = false;
/* 358 */         long agg_value31 = -1L;
/* 359 */         if (!false && agg_isNull21) {
/* 360 */           boolean agg_isNull31 = agg_fastAggBuffer.isNullAt(1);
/* 361 */           long agg_value33 = agg_isNull31 ?
/* 362 */           -1L : (agg_fastAggBuffer.getLong(1));
/* 363 */           agg_isNull29 = agg_isNull31;
/* 364 */           agg_value31 = agg_value33;
/* 365 */         } else {
/* 366 */           boolean agg_isNull32 = true;
/* 367 */           long agg_value34 = -1L;
/* 368 */
/* 369 */           boolean agg_isNull33 = agg_fastAggBuffer.isNullAt(1);
/* 370 */           long agg_value35 = agg_isNull33 ?
/* 371 */           -1L : (agg_fastAggBuffer.getLong(1));
/* 372 */           if (!agg_isNull33) {
/* 373 */             agg_isNull32 = false; // resultCode could change nullability.
/* 374 */             agg_value34 = agg_value35 + 1L;
/* 375 */
/* 376 */           }
/* 377 */           agg_isNull29 = agg_isNull32;
/* 378 */           agg_value31 = agg_value34;
/* 379 */         }
/* 380 */         // update fast row
/* 381 */         if (!agg_isNull23) {
/* 382 */           agg_fastAggBuffer.setDouble(0, agg_value25);
/* 383 */         } else {
/* 384 */           agg_fastAggBuffer.setNullAt(0);
/* 385 */         }
/* 386 */
/* 387 */         if (!agg_isNull29) {
/* 388 */           agg_fastAggBuffer.setLong(1, agg_value31);
/* 389 */         } else {
/* 390 */           agg_fastAggBuffer.setNullAt(1);
/* 391 */         }
/* 392 */       } else {
/* 393 */         // common sub-expressions
/* 394 */         boolean agg_isNull7 = false;
/* 395 */         long agg_value9 = -1L;
/* 396 */         if (!false) {
/* 397 */           agg_value9 = (long) agg_expr_0;
/* 398 */         }
/* 399 */         // evaluate aggregate function
/* 400 */         boolean agg_isNull9 = true;
/* 401 */         double agg_value11 = -1.0;
/* 402 */
/* 403 */         boolean agg_isNull10 = agg_unsafeRowAggBuffer.isNullAt(0);
/* 404 */         double agg_value12 = agg_isNull10 ?
/* 405 */         -1.0 : (agg_unsafeRowAggBuffer.getDouble(0));
/* 406 */         if (!agg_isNull10) {
/* 407 */           agg_agg_isNull11 = true;
/* 408 */           double agg_value13 = -1.0;
/* 409 */           do {
/* 410 */             boolean agg_isNull12 = agg_isNull7;
/* 411 */             double agg_value14 = -1.0;
/* 412 */             if (!agg_isNull7) {
/* 413 */               agg_value14 = (double) agg_value9;
/* 414 */             }
/* 415 */             if (!agg_isNull12) {
/* 416 */               agg_agg_isNull11 = false;
/* 417 */               agg_value13 = agg_value14;
/* 418 */               continue;
/* 419 */             }
/* 420 */
/* 421 */             boolean agg_isNull13 = false;
/* 422 */             double agg_value15 = -1.0;
/* 423 */             if (!false) {
/* 424 */               agg_value15 = (double) 0;
/* 425 */             }
/* 426 */             if (!agg_isNull13) {
/* 427 */               agg_agg_isNull11 = false;
/* 428 */               agg_value13 = agg_value15;
/* 429 */               continue;
/* 430 */             }
/* 431 */
/* 432 */           } while (false);
/* 433 */
/* 434 */           agg_isNull9 = false; // resultCode could change nullability.
/* 435 */           agg_value11 = agg_value12 + agg_value13;
/* 436 */
/* 437 */         }
/* 438 */         boolean agg_isNull15 = false;
/* 439 */         long agg_value17 = -1L;
/* 440 */         if (!false && agg_isNull7) {
/* 441 */           boolean agg_isNull17 = agg_unsafeRowAggBuffer.isNullAt(1);
/* 442 */           long agg_value19 = agg_isNull17 ?
/* 443 */           -1L : (agg_unsafeRowAggBuffer.getLong(1));
/* 444 */           agg_isNull15 = agg_isNull17;
/* 445 */           agg_value17 = agg_value19;
/* 446 */         } else {
/* 447 */           boolean agg_isNull18 = true;
/* 448 */           long agg_value20 = -1L;
/* 449 */
/* 450 */           boolean agg_isNull19 = agg_unsafeRowAggBuffer.isNullAt(1);
/* 451 */           long agg_value21 = agg_isNull19 ?
/* 452 */           -1L : (agg_unsafeRowAggBuffer.getLong(1));
/* 453 */           if (!agg_isNull19) {
/* 454 */             agg_isNull18 = false; // resultCode could change nullability.
/* 455 */             agg_value20 = agg_value21 + 1L;
/* 456 */
/* 457 */           }
/* 458 */           agg_isNull15 = agg_isNull18;
/* 459 */           agg_value17 = agg_value20;
/* 460 */         }
/* 461 */         // update unsafe row buffer
/* 462 */         if (!agg_isNull9) {
/* 463 */           agg_unsafeRowAggBuffer.setDouble(0, agg_value11);
/* 464 */         } else {
/* 465 */           agg_unsafeRowAggBuffer.setNullAt(0);
/* 466 */         }
/* 467 */
/* 468 */         if (!agg_isNull15) {
/* 469 */           agg_unsafeRowAggBuffer.setLong(1, agg_value17);
/* 470 */         } else {
/* 471 */           agg_unsafeRowAggBuffer.setNullAt(1);
/* 472 */         }
/* 473 */
/* 474 */       }
/* 475 */
/* 476 */     }
/* 477 */
/* 478 */   }
/* 479 */
/* 480 */ }
```

## How was this patch tested?

Added UT into `WholeStageCodegenSuite`

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

Closes #20779 from kiszk/SPARK-23598.
2018-03-13 23:04:16 +01:00
zuotingbing 918fb9beee [SPARK-23547][SQL] Cleanup the .pipeout file when the Hive Session closed
## What changes were proposed in this pull request?

![2018-03-07_121010](https://user-images.githubusercontent.com/24823338/37073232-922e10d2-2200-11e8-8172-6e03aa984b39.png)

when the hive session closed, we should also cleanup the .pipeout file.

## How was this patch tested?

Added test cases.

Author: zuotingbing <zuo.tingbing9@zte.com.cn>

Closes #20702 from zuotingbing/SPARK-23547.
2018-03-13 11:31:32 -07:00
Xingbo Jiang 9ddd1e2cea [MINOR][SQL][TEST] Create table using dataSourceName in HadoopFsRelationTest
## What changes were proposed in this pull request?

This PR fixes a minor issue in `HadoopFsRelationTest`, that you should create table using `dataSourceName` instead of `parquet`. The issue won't affect the correctness, but it will generate wrong error message in case the test fails.

## How was this patch tested?

Exsiting tests.

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

Closes #20780 from jiangxb1987/dataSourceName.
2018-03-13 23:31:08 +09: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
Wang Gengliang 10b0657b03 [SPARK-23624][SQL] Revise doc of method pushFilters in Datasource V2
## What changes were proposed in this pull request?

Revise doc of method pushFilters in SupportsPushDownFilters/SupportsPushDownCatalystFilters

In `FileSourceStrategy`, except `partitionKeyFilters`(the references of which is subset of partition keys), all filters needs to be evaluated after scanning. Otherwise, Spark will get wrong result from data sources like Orc/Parquet.

This PR is to improve the doc.

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

Closes #20769 from gengliangwang/revise_pushdown_doc.
2018-03-09 15:41:19 -08: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
Dilip Biswal d90e77bd0e [SPARK-23271][SQL] Parquet output contains only _SUCCESS file after writing an empty dataframe
## What changes were proposed in this pull request?
Below are the two cases.
``` SQL
case 1

scala> List.empty[String].toDF().rdd.partitions.length
res18: Int = 1
```
When we write the above data frame as parquet, we create a parquet file containing
just the schema of the data frame.

Case 2
``` SQL

scala> val anySchema = StructType(StructField("anyName", StringType, nullable = false) :: Nil)
anySchema: org.apache.spark.sql.types.StructType = StructType(StructField(anyName,StringType,false))
scala> spark.read.schema(anySchema).csv("/tmp/empty_folder").rdd.partitions.length
res22: Int = 0
```
For the 2nd case, since number of partitions = 0, we don't call the write task (the task has logic to create the empty metadata only parquet file)

The fix is to create a dummy single partition RDD and set up the write task based on it to ensure
the metadata-only file.

## How was this patch tested?

A new test is added to DataframeReaderWriterSuite.

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

Closes #20525 from dilipbiswal/spark-23271.
2018-03-08 14:58:40 -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
Li Jin 2cb23a8f51 [SPARK-23011][SQL][PYTHON] Support alternative function form with group aggregate pandas UDF
## What changes were proposed in this pull request?

This PR proposes to support an alternative function from with group aggregate pandas UDF.

The current form:
```
def foo(pdf):
    return ...
```
Takes a single arg that is a pandas DataFrame.

With this PR, an alternative form is supported:
```
def foo(key, pdf):
    return ...
```
The alternative form takes two argument - a tuple that presents the grouping key, and a pandas DataFrame represents the data.

## How was this patch tested?

GroupbyApplyTests

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

Closes #20295 from icexelloss/SPARK-23011-groupby-apply-key.
2018-03-08 20:29:07 +09: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
Xingbo Jiang ac76eff6a8 [SPARK-23525][SQL] Support ALTER TABLE CHANGE COLUMN COMMENT for external hive table
## What changes were proposed in this pull request?

The following query doesn't work as expected:
```
CREATE EXTERNAL TABLE ext_table(a STRING, b INT, c STRING) PARTITIONED BY (d STRING)
LOCATION 'sql/core/spark-warehouse/ext_table';
ALTER TABLE ext_table CHANGE a a STRING COMMENT "new comment";
DESC ext_table;
```
The comment of column `a` is not updated, that's because `HiveExternalCatalog.doAlterTable` ignores table schema changes. To fix the issue, we should call `doAlterTableDataSchema` instead of `doAlterTable`.

## How was this patch tested?

Updated `DDLSuite.testChangeColumn`.

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

Closes #20696 from jiangxb1987/alterColumnComment.
2018-03-07 13:51:44 -08:00
Marcelo Vanzin c99fc9ad9b [SPARK-23550][CORE] Cleanup Utils.
A few different things going on:
- Remove unused methods.
- Move JSON methods to the only class that uses them.
- Move test-only methods to TestUtils.
- Make getMaxResultSize() a config constant.
- Reuse functionality from existing libraries (JRE or JavaUtils) where possible.

The change also includes changes to a few tests to call `Utils.createTempFile` correctly,
so that temp dirs are created under the designated top-level temp dir instead of
potentially polluting git index.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20706 from vanzin/SPARK-23550.
2018-03-07 13:42:06 -08: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
Wenchen Fan ad640a5aff [SPARK-23303][SQL] improve the explain result for data source v2 relations
## What changes were proposed in this pull request?

The proposed explain format:
**[streaming header] [RelationV2/ScanV2] [data source name] [output] [pushed filters] [options]**

**streaming header**: if it's a streaming relation, put a "Streaming" at the beginning.
**RelationV2/ScanV2**: if it's a logical plan, put a "RelationV2", else, put a "ScanV2"
**data source name**: the simple class name of the data source implementation
**output**: a string of the plan output attributes
**pushed filters**: a string of all the filters that have been pushed to this data source
**options**: all the options to create the data source reader.

The current explain result for data source v2 relation is unreadable:
```
== Parsed Logical Plan ==
'Filter ('i > 6)
+- AnalysisBarrier
      +- Project [j#1]
         +- DataSourceV2Relation [i#0, j#1], org.apache.spark.sql.sources.v2.AdvancedDataSourceV2$Reader3b415940

== Analyzed Logical Plan ==
j: int
Project [j#1]
+- Filter (i#0 > 6)
   +- Project [j#1, i#0]
      +- DataSourceV2Relation [i#0, j#1], org.apache.spark.sql.sources.v2.AdvancedDataSourceV2$Reader3b415940

== Optimized Logical Plan ==
Project [j#1]
+- Filter isnotnull(i#0)
   +- DataSourceV2Relation [i#0, j#1], org.apache.spark.sql.sources.v2.AdvancedDataSourceV2$Reader3b415940

== Physical Plan ==
*(1) Project [j#1]
+- *(1) Filter isnotnull(i#0)
   +- *(1) DataSourceV2Scan [i#0, j#1], org.apache.spark.sql.sources.v2.AdvancedDataSourceV2$Reader3b415940
```

after this PR
```
== Parsed Logical Plan ==
'Project [unresolvedalias('j, None)]
+- AnalysisBarrier
      +- RelationV2 AdvancedDataSourceV2[i#0, j#1]

== Analyzed Logical Plan ==
j: int
Project [j#1]
+- RelationV2 AdvancedDataSourceV2[i#0, j#1]

== Optimized Logical Plan ==
RelationV2 AdvancedDataSourceV2[j#1]

== Physical Plan ==
*(1) ScanV2 AdvancedDataSourceV2[j#1]
```
-------
```
== Analyzed Logical Plan ==
i: int, j: int
Filter (i#88 > 3)
+- RelationV2 JavaAdvancedDataSourceV2[i#88, j#89]

== Optimized Logical Plan ==
Filter isnotnull(i#88)
+- RelationV2 JavaAdvancedDataSourceV2[i#88, j#89] (Pushed Filters: [GreaterThan(i,3)])

== Physical Plan ==
*(1) Filter isnotnull(i#88)
+- *(1) ScanV2 JavaAdvancedDataSourceV2[i#88, j#89] (Pushed Filters: [GreaterThan(i,3)])
```

an example for streaming query
```
== Parsed Logical Plan ==
Aggregate [value#6], [value#6, count(1) AS count(1)#11L]
+- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#6]
   +- MapElements <function1>, class java.lang.String, [StructField(value,StringType,true)], obj#5: java.lang.String
      +- DeserializeToObject cast(value#25 as string).toString, obj#4: java.lang.String
         +- Streaming RelationV2 MemoryStreamDataSource[value#25]

== Analyzed Logical Plan ==
value: string, count(1): bigint
Aggregate [value#6], [value#6, count(1) AS count(1)#11L]
+- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#6]
   +- MapElements <function1>, class java.lang.String, [StructField(value,StringType,true)], obj#5: java.lang.String
      +- DeserializeToObject cast(value#25 as string).toString, obj#4: java.lang.String
         +- Streaming RelationV2 MemoryStreamDataSource[value#25]

== Optimized Logical Plan ==
Aggregate [value#6], [value#6, count(1) AS count(1)#11L]
+- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#6]
   +- MapElements <function1>, class java.lang.String, [StructField(value,StringType,true)], obj#5: java.lang.String
      +- DeserializeToObject value#25.toString, obj#4: java.lang.String
         +- Streaming RelationV2 MemoryStreamDataSource[value#25]

== Physical Plan ==
*(4) HashAggregate(keys=[value#6], functions=[count(1)], output=[value#6, count(1)#11L])
+- StateStoreSave [value#6], state info [ checkpoint = *********(redacted)/cloud/dev/spark/target/tmp/temporary-549f264b-2531-4fcb-a52f-433c77347c12/state, runId = f84d9da9-2f8c-45c1-9ea1-70791be684de, opId = 0, ver = 0, numPartitions = 5], Complete, 0
   +- *(3) HashAggregate(keys=[value#6], functions=[merge_count(1)], output=[value#6, count#16L])
      +- StateStoreRestore [value#6], state info [ checkpoint = *********(redacted)/cloud/dev/spark/target/tmp/temporary-549f264b-2531-4fcb-a52f-433c77347c12/state, runId = f84d9da9-2f8c-45c1-9ea1-70791be684de, opId = 0, ver = 0, numPartitions = 5]
         +- *(2) HashAggregate(keys=[value#6], functions=[merge_count(1)], output=[value#6, count#16L])
            +- Exchange hashpartitioning(value#6, 5)
               +- *(1) HashAggregate(keys=[value#6], functions=[partial_count(1)], output=[value#6, count#16L])
                  +- *(1) SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#6]
                     +- *(1) MapElements <function1>, obj#5: java.lang.String
                        +- *(1) DeserializeToObject value#25.toString, obj#4: java.lang.String
                           +- *(1) ScanV2 MemoryStreamDataSource[value#25]
```
## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20647 from cloud-fan/explain.
2018-03-05 20:35:14 -08:00
Henry Robinson 8c5b34c425 [SPARK-23604][SQL] Change Statistics.isEmpty to !Statistics.hasNonNul…
…lValue

## What changes were proposed in this pull request?

Parquet 1.9 will change the semantics of Statistics.isEmpty slightly
to reflect if the null value count has been set. That breaks a
timestamp interoperability test that cares only about whether there
are column values present in the statistics of a written file for an
INT96 column. Fix by using Statistics.hasNonNullValue instead.

## How was this patch tested?

Unit tests continue to pass against Parquet 1.8, and also pass against
a Parquet build including PARQUET-1217.

Author: Henry Robinson <henry@cloudera.com>

Closes #20740 from henryr/spark-23604.
2018-03-05 16:49:24 -08: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
Jose Torres b0f422c386 [SPARK-23559][SS] Add epoch ID to DataWriterFactory.
## What changes were proposed in this pull request?

Add an epoch ID argument to DataWriterFactory for use in streaming. As a side effect of passing in this value, DataWriter will now have a consistent lifecycle; commit() or abort() ends the lifecycle of a DataWriter instance in any execution mode.

I considered making a separate streaming interface and adding the epoch ID only to that one, but I think it requires a lot of extra work for no real gain. I think it makes sense to define epoch 0 as the one and only epoch of a non-streaming query.

## How was this patch tested?

existing unit tests

Author: Jose Torres <jose@databricks.com>

Closes #20710 from jose-torres/api2.
2018-03-05 13:23:01 -08: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
Juliusz Sompolski dea381dfaa [SPARK-23514][FOLLOW-UP] Remove more places using sparkContext.hadoopConfiguration directly
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/20679 I missed a few places in SQL tests.
For hygiene, they should also use the sessionState interface where possible.

## How was this patch tested?

Modified existing tests.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #20718 from juliuszsompolski/SPARK-23514-followup.
2018-03-03 09:10:48 +08:00
gatorsmile 487377e693 [SPARK-23570][SQL] Add Spark 2.3.0 in HiveExternalCatalogVersionsSuite
## What changes were proposed in this pull request?
Add Spark 2.3.0 in HiveExternalCatalogVersionsSuite since Spark 2.3.0 is released for ensuring backward compatibility.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20720 from gatorsmile/add2.3.
2018-03-02 14:30:37 -08:00
jerryshao 707e6506d0 [SPARK-23097][SQL][SS] Migrate text socket source to V2
## What changes were proposed in this pull request?

This PR moves structured streaming text socket source to V2.

Questions: do we need to remove old "socket" source?

## How was this patch tested?

Unit test and manual verification.

Author: jerryshao <sshao@hortonworks.com>

Closes #20382 from jerryshao/SPARK-23097.
2018-03-02 12:27:42 -08: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
Yuming Wang ff1480189b [SPARK-23510][SQL] Support Hive 2.2 and Hive 2.3 metastore
## What changes were proposed in this pull request?
This is based on https://github.com/apache/spark/pull/20668 for supporting Hive 2.2 and Hive 2.3 metastore.

When we merge the PR, we should give the major credit to wangyum

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

Author: Yuming Wang <yumwang@ebay.com>
Author: gatorsmile <gatorsmile@gmail.com>

Closes #20671 from gatorsmile/pr-20668.
2018-03-01 16:26:11 +08:00
Xingbo Jiang 25c2776dd9 [SPARK-23523][SQL][FOLLOWUP] Minor refactor of OptimizeMetadataOnlyQuery
## What changes were proposed in this pull request?

Inside `OptimizeMetadataOnlyQuery.getPartitionAttrs`, avoid using `zip` to generate attribute map.
Also include other minor update of comments and format.

## How was this patch tested?

Existing test cases.

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

Closes #20693 from jiangxb1987/SPARK-23523.
2018-02-28 12:16:26 -08:00
Juliusz Sompolski 476a7f026b [SPARK-23514] Use SessionState.newHadoopConf() to propage hadoop configs set in SQLConf.
## What changes were proposed in this pull request?

A few places in `spark-sql` were using `sc.hadoopConfiguration` directly. They should be using `sessionState.newHadoopConf()` to blend in configs that were set through `SQLConf`.

Also, for better UX, for these configs blended in from `SQLConf`, we should consider removing the `spark.hadoop` prefix, so that the settings are recognized whether or not they were specified by the user.

## How was this patch tested?

Tested that AlterTableRecoverPartitions now correctly recognizes settings that are passed in to the FileSystem through SQLConf.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #20679 from juliuszsompolski/SPARK-23514.
2018-02-28 08:44:53 -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
Jose Torres 7ec83658fb [SPARK-23491][SS] Remove explicit job cancellation from ContinuousExecution reconfiguring
## What changes were proposed in this pull request?

Remove queryExecutionThread.interrupt() from ContinuousExecution. As detailed in the JIRA, interrupting the thread is only relevant in the microbatch case; for continuous processing the query execution can quickly clean itself up without.

## How was this patch tested?

existing tests

Author: Jose Torres <jose@databricks.com>

Closes #20622 from jose-torres/SPARK-23441.
2018-02-26 11:28:44 -08:00
Kazuaki Ishizaki 1a198ce8f5 [SPARK-23459][SQL] Improve the error message when unknown column is specified in partition columns
## What changes were proposed in this pull request?

This PR avoids to print schema internal information when unknown column is specified in partition columns. This PR prints column names in the schema with more readable format.

The following is an example.

Source code
```
test("save with an unknown partition column") {
  withTempDir { dir =>
    val path = dir.getCanonicalPath
      Seq(1L -> "a").toDF("i", "j").write
        .format("parquet")
        .partitionBy("unknownColumn")
        .save(path)
  }
```
Output without this PR
```
Partition column unknownColumn not found in schema StructType(StructField(i,LongType,false), StructField(j,StringType,true));
```

Output with this PR
```
Partition column unknownColumn not found in schema struct<i:bigint,j:string>;
```

## How was this patch tested?

Manually tested

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

Closes #20653 from kiszk/SPARK-23459.
2018-02-23 16:30:32 -08:00
Tathagata Das 855ce13d04 [SPARK-23408][SS] Synchronize successive AddData actions in Streaming*JoinSuite
**The best way to review this PR is to ignore whitespace/indent changes. Use this link - https://github.com/apache/spark/pull/20650/files?w=1**

## What changes were proposed in this pull request?

The stream-stream join tests add data to multiple sources and expect it all to show up in the next batch. But there's a race condition; the new batch might trigger when only one of the AddData actions has been reached.

Prior attempt to solve this issue by jose-torres in #20646 attempted to simultaneously synchronize on all memory sources together when consecutive AddData was found in the actions. However, this carries the risk of deadlock as well as unintended modification of stress tests (see the above PR for a detailed explanation). Instead, this PR attempts the following.

- A new action called `StreamProgressBlockedActions` that allows multiple actions to be executed while the streaming query is blocked from making progress. This allows data to be added to multiple sources that are made visible simultaneously in the next batch.
- An alias of `StreamProgressBlockedActions` called `MultiAddData` is explicitly used in the `Streaming*JoinSuites` to add data to two memory sources simultaneously.

This should avoid unintentional modification of the stress tests (or any other test for that matter) while making sure that the flaky tests are deterministic.

## How was this patch tested?
Modified test cases in `Streaming*JoinSuites` where there are consecutive `AddData` actions.

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

Closes #20650 from tdas/SPARK-23408.
2018-02-23 12:40:58 -08:00
Wang Gengliang 049f243c59 [SPARK-23490][SQL] Check storage.locationUri with existing table in CreateTable
## What changes were proposed in this pull request?

For CreateTable with Append mode, we should check if `storage.locationUri` is the same with existing table in `PreprocessTableCreation`

In the current code, there is only a simple exception if the `storage.locationUri` is different with existing table:
`org.apache.spark.sql.AnalysisException: Table or view not found:`

which can be improved.

## How was this patch tested?

Unit test

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

Closes #20660 from gengliangwang/locationUri.
2018-02-22 21:49:25 -08:00
Ryan Blue c8c4441dfd [SPARK-23418][SQL] Fail DataSourceV2 reads when user schema is passed, but not supported.
## What changes were proposed in this pull request?

DataSourceV2 initially allowed user-supplied schemas when a source doesn't implement `ReadSupportWithSchema`, as long as the schema was identical to the source's schema. This is confusing behavior because changes to an underlying table can cause a previously working job to fail with an exception that user-supplied schemas are not allowed.

This reverts commit adcb25a0624, which was added to #20387 so that it could be removed in a separate JIRA issue and PR.

## How was this patch tested?

Existing tests.

Author: Ryan Blue <blue@apache.org>

Closes #20603 from rdblue/SPARK-23418-revert-adcb25a0624.
2018-02-21 15:10:08 +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 3e48f3b9ee [SPARK-23434][SQL] Spark should not warn metadata directory for a HDFS file path
## What changes were proposed in this pull request?

In a kerberized cluster, when Spark reads a file path (e.g. `people.json`), it warns with a wrong warning message during looking up `people.json/_spark_metadata`. The root cause of this situation is the difference between `LocalFileSystem` and `DistributedFileSystem`. `LocalFileSystem.exists()` returns `false`, but `DistributedFileSystem.exists` raises `org.apache.hadoop.security.AccessControlException`.

```scala
scala> spark.version
res0: String = 2.4.0-SNAPSHOT

scala> spark.read.json("file:///usr/hdp/current/spark-client/examples/src/main/resources/people.json").show
+----+-------+
| age|   name|
+----+-------+
|null|Michael|
|  30|   Andy|
|  19| Justin|
+----+-------+

scala> spark.read.json("hdfs:///tmp/people.json")
18/02/15 05:00:48 WARN streaming.FileStreamSink: Error while looking for metadata directory.
18/02/15 05:00:48 WARN streaming.FileStreamSink: Error while looking for metadata directory.
```

After this PR,
```scala
scala> spark.read.json("hdfs:///tmp/people.json").show
+----+-------+
| age|   name|
+----+-------+
|null|Michael|
|  30|   Andy|
|  19| Justin|
+----+-------+
```

## How was this patch tested?

Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20616 from dongjoon-hyun/SPARK-23434.
2018-02-20 16:02:44 -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
Ryan Blue aadf9535b4 [SPARK-23203][SQL] DataSourceV2: Use immutable logical plans.
## What changes were proposed in this pull request?

SPARK-23203: DataSourceV2 should use immutable catalyst trees instead of wrapping a mutable DataSourceV2Reader. This commit updates DataSourceV2Relation and consolidates much of the DataSourceV2 API requirements for the read path in it. Instead of wrapping a reader that changes, the relation lazily produces a reader from its configuration.

This commit also updates the predicate and projection push-down. Instead of the implementation from SPARK-22197, this reuses the rule matching from the Hive and DataSource read paths (using `PhysicalOperation`) and copies most of the implementation of `SparkPlanner.pruneFilterProject`, with updates for DataSourceV2. By reusing the implementation from other read paths, this should have fewer regressions from other read paths and is less code to maintain.

The new push-down rules also supports the following edge cases:

* The output of DataSourceV2Relation should be what is returned by the reader, in case the reader can only partially satisfy the requested schema projection
* The requested projection passed to the DataSourceV2Reader should include filter columns
* The push-down rule may be run more than once if filters are not pushed through projections

## How was this patch tested?

Existing push-down and read tests.

Author: Ryan Blue <blue@apache.org>

Closes #20387 from rdblue/SPARK-22386-push-down-immutable-trees.
2018-02-20 16:04:22 +08:00
Marco Gaido 651b0277fe [SPARK-23436][SQL] Infer partition as Date only if it can be casted to Date
## What changes were proposed in this pull request?

Before the patch, Spark could infer as Date a partition value which cannot be casted to Date (this can happen when there are extra characters after a valid date, like `2018-02-15AAA`).

When this happens and the input format has metadata which define the schema of the table, then `null` is returned as a value for the partition column, because the `cast` operator used in (`PartitioningAwareFileIndex.inferPartitioning`) is unable to convert the value.

The PR checks in the partition inference that values can be casted to Date and Timestamp, in order to infer that datatype to them.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20621 from mgaido91/SPARK-23436.
2018-02-20 13:56:38 +08:00
Dongjoon Hyun f5850e7892 [SPARK-23457][SQL] Register task completion listeners first in ParquetFileFormat
## What changes were proposed in this pull request?

ParquetFileFormat leaks opened files in some cases. This PR prevents that by registering task completion listers first before initialization.

- [spark-branch-2.3-test-sbt-hadoop-2.7](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-branch-2.3-test-sbt-hadoop-2.7/205/testReport/org.apache.spark.sql/FileBasedDataSourceSuite/_It_is_not_a_test_it_is_a_sbt_testing_SuiteSelector_/)
- [spark-master-test-sbt-hadoop-2.6](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.6/4228/testReport/junit/org.apache.spark.sql.execution.datasources.parquet/ParquetQuerySuite/_It_is_not_a_test_it_is_a_sbt_testing_SuiteSelector_/)

```
Caused by: sbt.ForkMain$ForkError: java.lang.Throwable: null
	at org.apache.spark.DebugFilesystem$.addOpenStream(DebugFilesystem.scala:36)
	at org.apache.spark.DebugFilesystem.open(DebugFilesystem.scala:70)
	at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:769)
	at org.apache.parquet.hadoop.ParquetFileReader.<init>(ParquetFileReader.java:538)
	at org.apache.spark.sql.execution.datasources.parquet.SpecificParquetRecordReaderBase.initialize(SpecificParquetRecordReaderBase.java:149)
	at org.apache.spark.sql.execution.datasources.parquet.VectorizedParquetRecordReader.initialize(VectorizedParquetRecordReader.java:133)
	at org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$$anonfun$buildReaderWithPartitionValues$1.apply(ParquetFileFormat.scala:400)
	at
```

## How was this patch tested?

Manual. The following test case generates the same leakage.

```scala
  test("SPARK-23457 Register task completion listeners first in ParquetFileFormat") {
    withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_BATCH_SIZE.key -> s"${Int.MaxValue}") {
      withTempDir { dir =>
        val basePath = dir.getCanonicalPath
        Seq(0).toDF("a").write.format("parquet").save(new Path(basePath, "first").toString)
        Seq(1).toDF("a").write.format("parquet").save(new Path(basePath, "second").toString)
        val df = spark.read.parquet(
          new Path(basePath, "first").toString,
          new Path(basePath, "second").toString)
        val e = intercept[SparkException] {
          df.collect()
        }
        assert(e.getCause.isInstanceOf[OutOfMemoryError])
      }
    }
  }
```

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20619 from dongjoon-hyun/SPARK-23390.
2018-02-20 13:33:03 +08:00
Dongjoon Hyun 3ee3b2ae1f [SPARK-23340][SQL] Upgrade Apache ORC to 1.4.3
## What changes were proposed in this pull request?

This PR updates Apache ORC dependencies to 1.4.3 released on February 9th. Apache ORC 1.4.2 release removes unnecessary dependencies and 1.4.3 has 5 more patches (https://s.apache.org/Fll8).

Especially, the following ORC-285 is fixed at 1.4.3.

```scala
scala> val df = Seq(Array.empty[Float]).toDF()

scala> df.write.format("orc").save("/tmp/floatarray")

scala> spark.read.orc("/tmp/floatarray")
res1: org.apache.spark.sql.DataFrame = [value: array<float>]

scala> spark.read.orc("/tmp/floatarray").show()
18/02/12 22:09:10 ERROR Executor: Exception in task 0.0 in stage 1.0 (TID 1)
java.io.IOException: Error reading file: file:/tmp/floatarray/part-00000-9c0b461b-4df1-4c23-aac1-3e4f349ac7d6-c000.snappy.orc
	at org.apache.orc.impl.RecordReaderImpl.nextBatch(RecordReaderImpl.java:1191)
	at org.apache.orc.mapreduce.OrcMapreduceRecordReader.ensureBatch(OrcMapreduceRecordReader.java:78)
...
Caused by: java.io.EOFException: Read past EOF for compressed stream Stream for column 2 kind DATA position: 0 length: 0 range: 0 offset: 0 limit: 0
```

## How was this patch tested?

Pass the Jenkins test.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20511 from dongjoon-hyun/SPARK-23340.
2018-02-17 00:25:36 -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 6968c3cfd7 [MINOR][SQL] Fix an error message about inserting into bucketed tables
## What changes were proposed in this pull request?

This replaces `Sparkcurrently` to `Spark currently` in the following error message.

```scala
scala> sql("insert into t2 select * from v1")
org.apache.spark.sql.AnalysisException: Output Hive table `default`.`t2`
is bucketed but Sparkcurrently does NOT populate bucketed ...
```

## How was this patch tested?

Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20617 from dongjoon-hyun/SPARK-ERROR-MSG.
2018-02-15 09:40:08 -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
Wenchen Fan f38c760638 [SPARK-23419][SPARK-23416][SS] data source v2 write path should re-throw interruption exceptions directly
## What changes were proposed in this pull request?

Streaming execution has a list of exceptions that means interruption, and handle them specially. `WriteToDataSourceV2Exec` should also respect this list and not wrap them with `SparkException`.

## How was this patch tested?

existing test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20605 from cloud-fan/write.
2018-02-15 16:59:44 +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 a77ebb0921 [SPARK-23421][SPARK-22356][SQL] Document the behavior change in
## What changes were proposed in this pull request?
https://github.com/apache/spark/pull/19579 introduces a behavior change. We need to document it in the migration guide.

## How was this patch tested?
Also update the HiveExternalCatalogVersionsSuite to verify it.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20606 from gatorsmile/addMigrationGuide.
2018-02-14 23:52:59 -08:00
Tathagata Das 658d9d9d78 [SPARK-23406][SS] Enable stream-stream self-joins
## What changes were proposed in this pull request?

Solved two bugs to enable stream-stream self joins.

### Incorrect analysis due to missing MultiInstanceRelation trait
Streaming leaf nodes did not extend MultiInstanceRelation, which is necessary for the catalyst analyzer to convert the self-join logical plan DAG into a tree (by creating new instances of the leaf relations). This was causing the error `Failure when resolving conflicting references in Join:` (see JIRA for details).

### Incorrect attribute rewrite when splicing batch plans in MicroBatchExecution
When splicing the source's batch plan into the streaming plan (by replacing the StreamingExecutionPlan), we were rewriting the attribute reference in the streaming plan with the new attribute references from the batch plan. This was incorrectly handling the scenario when multiple StreamingExecutionRelation point to the same source, and therefore eventually point to the same batch plan returned by the source. Here is an example query, and its corresponding plan transformations.
```
val df = input.toDF
val join =
      df.select('value % 5 as "key", 'value).join(
        df.select('value % 5 as "key", 'value), "key")
```
Streaming logical plan before splicing the batch plan
```
Project [key#6, value#1, value#12]
+- Join Inner, (key#6 = key#9)
   :- Project [(value#1 % 5) AS key#6, value#1]
   :  +- StreamingExecutionRelation Memory[#1], value#1
   +- Project [(value#12 % 5) AS key#9, value#12]
      +- StreamingExecutionRelation Memory[#1], value#12  // two different leaves pointing to same source
```
Batch logical plan after splicing the batch plan and before rewriting
```
Project [key#6, value#1, value#12]
+- Join Inner, (key#6 = key#9)
   :- Project [(value#1 % 5) AS key#6, value#1]
   :  +- LocalRelation [value#66]           // replaces StreamingExecutionRelation Memory[#1], value#1
   +- Project [(value#12 % 5) AS key#9, value#12]
      +- LocalRelation [value#66]           // replaces StreamingExecutionRelation Memory[#1], value#12
```
Batch logical plan after rewriting the attributes. Specifically, for spliced, the new output attributes (value#66) replace the earlier output attributes (value#12, and value#1, one for each StreamingExecutionRelation).
```
Project [key#6, value#66, value#66]       // both value#1 and value#12 replaces by value#66
+- Join Inner, (key#6 = key#9)
   :- Project [(value#66 % 5) AS key#6, value#66]
   :  +- LocalRelation [value#66]
   +- Project [(value#66 % 5) AS key#9, value#66]
      +- LocalRelation [value#66]
```
This causes the optimizer to eliminate value#66 from one side of the join.
```
Project [key#6, value#66, value#66]
+- Join Inner, (key#6 = key#9)
   :- Project [(value#66 % 5) AS key#6, value#66]
   :  +- LocalRelation [value#66]
   +- Project [(value#66 % 5) AS key#9]   // this does not generate value, incorrect join results
      +- LocalRelation [value#66]
```

**Solution**: Instead of rewriting attributes, use a Project to introduce aliases between the output attribute references and the new reference generated by the spliced plans. The analyzer and optimizer will take care of the rest.
```
Project [key#6, value#1, value#12]
+- Join Inner, (key#6 = key#9)
   :- Project [(value#1 % 5) AS key#6, value#1]
   :  +- Project [value#66 AS value#1]   // solution: project with aliases
   :     +- LocalRelation [value#66]
   +- Project [(value#12 % 5) AS key#9, value#12]
      +- Project [value#66 AS value#12]    // solution: project with aliases
         +- LocalRelation [value#66]
```

## How was this patch tested?
New unit test

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

Closes #20598 from tdas/SPARK-23406.
2018-02-14 14:27:02 -08:00
gatorsmile 400a1d9e25 Revert "[SPARK-23249][SQL] Improved block merging logic for partitions"
This reverts commit 8c21170dec.
2018-02-14 10:57:12 -08:00
Dongjoon Hyun 357babde5a [SPARK-23399][SQL] Register a task completion listener first for OrcColumnarBatchReader
## What changes were proposed in this pull request?

This PR aims to resolve an open file leakage issue reported at [SPARK-23390](https://issues.apache.org/jira/browse/SPARK-23390) by moving the listener registration position. Currently, the sequence is like the following.

1. Create `batchReader`
2. `batchReader.initialize` opens a ORC file.
3. `batchReader.initBatch` may take a long time to alloc memory in some environment and cause errors.
4. `Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => iter.close()))`

This PR moves 4 before 2 and 3. To sum up, the new sequence is 1 -> 4 -> 2 -> 3.

## How was this patch tested?

Manual. The following test case makes OOM intentionally to cause leaked filesystem connection in the current code base. With this patch, leakage doesn't occurs.

```scala
  // This should be tested manually because it raises OOM intentionally
  // in order to cause `Leaked filesystem connection`.
  test("SPARK-23399 Register a task completion listener first for OrcColumnarBatchReader") {
    withSQLConf(SQLConf.ORC_VECTORIZED_READER_BATCH_SIZE.key -> s"${Int.MaxValue}") {
      withTempDir { dir =>
        val basePath = dir.getCanonicalPath
        Seq(0).toDF("a").write.format("orc").save(new Path(basePath, "first").toString)
        Seq(1).toDF("a").write.format("orc").save(new Path(basePath, "second").toString)
        val df = spark.read.orc(
          new Path(basePath, "first").toString,
          new Path(basePath, "second").toString)
        val e = intercept[SparkException] {
          df.collect()
        }
        assert(e.getCause.isInstanceOf[OutOfMemoryError])
      }
    }
  }
```

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20590 from dongjoon-hyun/SPARK-23399.
2018-02-14 10:55:24 +08:00
gatorsmile d6f5e172b4 Revert "[SPARK-23303][SQL] improve the explain result for data source v2 relations"
This reverts commit f17b936f0d.
2018-02-13 16:21:17 -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
Wenchen Fan f17b936f0d [SPARK-23303][SQL] improve the explain result for data source v2 relations
## What changes were proposed in this pull request?

The current explain result for data source v2 relation is unreadable:
```
== Parsed Logical Plan ==
'Filter ('i > 6)
+- AnalysisBarrier
      +- Project [j#1]
         +- DataSourceV2Relation [i#0, j#1], org.apache.spark.sql.sources.v2.AdvancedDataSourceV2$Reader3b415940

== Analyzed Logical Plan ==
j: int
Project [j#1]
+- Filter (i#0 > 6)
   +- Project [j#1, i#0]
      +- DataSourceV2Relation [i#0, j#1], org.apache.spark.sql.sources.v2.AdvancedDataSourceV2$Reader3b415940

== Optimized Logical Plan ==
Project [j#1]
+- Filter isnotnull(i#0)
   +- DataSourceV2Relation [i#0, j#1], org.apache.spark.sql.sources.v2.AdvancedDataSourceV2$Reader3b415940

== Physical Plan ==
*(1) Project [j#1]
+- *(1) Filter isnotnull(i#0)
   +- *(1) DataSourceV2Scan [i#0, j#1], org.apache.spark.sql.sources.v2.AdvancedDataSourceV2$Reader3b415940
```

after this PR
```
== Parsed Logical Plan ==
'Project [unresolvedalias('j, None)]
+- AnalysisBarrier
      +- Relation AdvancedDataSourceV2[i#0, j#1]

== Analyzed Logical Plan ==
j: int
Project [j#1]
+- Relation AdvancedDataSourceV2[i#0, j#1]

== Optimized Logical Plan ==
Relation AdvancedDataSourceV2[j#1]

== Physical Plan ==
*(1) Scan AdvancedDataSourceV2[j#1]
```
-------
```
== Analyzed Logical Plan ==
i: int, j: int
Filter (i#88 > 3)
+- Relation JavaAdvancedDataSourceV2[i#88, j#89]

== Optimized Logical Plan ==
Filter isnotnull(i#88)
+- Relation JavaAdvancedDataSourceV2[i#88, j#89] (PushedFilter: [GreaterThan(i,3)])

== Physical Plan ==
*(1) Filter isnotnull(i#88)
+- *(1) Scan JavaAdvancedDataSourceV2[i#88, j#89] (PushedFilter: [GreaterThan(i,3)])
```

an example for streaming query
```
== Parsed Logical Plan ==
Aggregate [value#6], [value#6, count(1) AS count(1)#11L]
+- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#6]
   +- MapElements <function1>, class java.lang.String, [StructField(value,StringType,true)], obj#5: java.lang.String
      +- DeserializeToObject cast(value#25 as string).toString, obj#4: java.lang.String
         +- Streaming Relation FakeDataSourceV2$[value#25]

== Analyzed Logical Plan ==
value: string, count(1): bigint
Aggregate [value#6], [value#6, count(1) AS count(1)#11L]
+- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#6]
   +- MapElements <function1>, class java.lang.String, [StructField(value,StringType,true)], obj#5: java.lang.String
      +- DeserializeToObject cast(value#25 as string).toString, obj#4: java.lang.String
         +- Streaming Relation FakeDataSourceV2$[value#25]

== Optimized Logical Plan ==
Aggregate [value#6], [value#6, count(1) AS count(1)#11L]
+- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#6]
   +- MapElements <function1>, class java.lang.String, [StructField(value,StringType,true)], obj#5: java.lang.String
      +- DeserializeToObject value#25.toString, obj#4: java.lang.String
         +- Streaming Relation FakeDataSourceV2$[value#25]

== Physical Plan ==
*(4) HashAggregate(keys=[value#6], functions=[count(1)], output=[value#6, count(1)#11L])
+- StateStoreSave [value#6], state info [ checkpoint = *********(redacted)/cloud/dev/spark/target/tmp/temporary-549f264b-2531-4fcb-a52f-433c77347c12/state, runId = f84d9da9-2f8c-45c1-9ea1-70791be684de, opId = 0, ver = 0, numPartitions = 5], Complete, 0
   +- *(3) HashAggregate(keys=[value#6], functions=[merge_count(1)], output=[value#6, count#16L])
      +- StateStoreRestore [value#6], state info [ checkpoint = *********(redacted)/cloud/dev/spark/target/tmp/temporary-549f264b-2531-4fcb-a52f-433c77347c12/state, runId = f84d9da9-2f8c-45c1-9ea1-70791be684de, opId = 0, ver = 0, numPartitions = 5]
         +- *(2) HashAggregate(keys=[value#6], functions=[merge_count(1)], output=[value#6, count#16L])
            +- Exchange hashpartitioning(value#6, 5)
               +- *(1) HashAggregate(keys=[value#6], functions=[partial_count(1)], output=[value#6, count#16L])
                  +- *(1) SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#6]
                     +- *(1) MapElements <function1>, obj#5: java.lang.String
                        +- *(1) DeserializeToObject value#25.toString, obj#4: java.lang.String
                           +- *(1) Scan FakeDataSourceV2$[value#25]
```
## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20477 from cloud-fan/explain.
2018-02-12 21:12:22 -08:00
Feng Liu ed4e78bd60 [SPARK-23379][SQL] skip when setting the same current database in HiveClientImpl
## What changes were proposed in this pull request?

If the target database name is as same as the current database, we should be able to skip one metastore access.

## 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: Feng Liu <fengliu@databricks.com>

Closes #20565 from liufengdb/remove-redundant.
2018-02-12 20:57:26 -08:00
Ryan Blue c1bcef876c [SPARK-23323][SQL] Support commit coordinator for DataSourceV2 writes
## What changes were proposed in this pull request?

DataSourceV2 batch writes should use the output commit coordinator if it is required by the data source. This adds a new method, `DataWriterFactory#useCommitCoordinator`, that determines whether the coordinator will be used. If the write factory returns true, `WriteToDataSourceV2` will use the coordinator for batch writes.

## How was this patch tested?

This relies on existing write tests, which now use the commit coordinator.

Author: Ryan Blue <blue@apache.org>

Closes #20490 from rdblue/SPARK-23323-add-commit-coordinator.
2018-02-13 11:40:34 +08:00
sychen 4104b68e95 [SPARK-23230][SQL] When hive.default.fileformat is other kinds of file types, create textfile table cause a serde error
When hive.default.fileformat is other kinds of file types, create textfile table cause a serde error.
We should take the default type of textfile and sequencefile both as org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe.

```
set hive.default.fileformat=orc;
create table tbl( i string ) stored as textfile;
desc formatted tbl;

Serde Library org.apache.hadoop.hive.ql.io.orc.OrcSerde
InputFormat  org.apache.hadoop.mapred.TextInputFormat
OutputFormat  org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
```

Author: sychen <sychen@ctrip.com>

Closes #20406 from cxzl25/default_serde.
2018-02-12 16:00:47 -08:00
Feng Liu fba01b9a65 [SPARK-23378][SQL] move setCurrentDatabase from HiveExternalCatalog to HiveClientImpl
## What changes were proposed in this pull request?

This removes the special case that `alterPartitions` call from `HiveExternalCatalog` can reset the current database in the hive client as a side effect.

## 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: Feng Liu <fengliu@databricks.com>

Closes #20564 from liufengdb/move.
2018-02-12 14:58:31 -08:00
Takuya UESHIN 0c66fe4f22 [SPARK-22002][SQL][FOLLOWUP][TEST] Add a test to check if the original schema doesn't have metadata.
## What changes were proposed in this pull request?

This is a follow-up pr of #19231 which modified the behavior to remove metadata from JDBC table schema.
This pr adds a test to check if the schema doesn't have metadata.

## How was this patch tested?

Added a test and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #20585 from ueshin/issues/SPARK-22002/fup1.
2018-02-12 12:20:29 -08:00
James Thompson 5bb11411ae [SPARK-23388][SQL] Support for Parquet Binary DecimalType in VectorizedColumnReader
## What changes were proposed in this pull request?

Re-add support for parquet binary DecimalType in VectorizedColumnReader

## How was this patch tested?

Existing test suite

Author: James Thompson <jamesthomp@users.noreply.github.com>

Closes #20580 from jamesthomp/jt/add-back-binary-decimal.
2018-02-12 11:34:56 -08:00
liuxian 4a4dd4f36f [SPARK-23391][CORE] It may lead to overflow for some integer multiplication
## What changes were proposed in this pull request?
In the `getBlockData`,`blockId.reduceId` is the `Int` type, when it is greater than 2^28, `blockId.reduceId*8` will overflow
In the `decompress0`, `len` and  `unitSize` are  Int type, so `len * unitSize` may lead to  overflow
## How was this patch tested?
N/A

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

Closes #20581 from 10110346/overflow2.
2018-02-12 08:49:45 -06:00
Wenchen Fan 0e2c266de7 [SPARK-22977][SQL] fix web UI SQL tab for CTAS
## What changes were proposed in this pull request?

This is a regression in Spark 2.3.

In Spark 2.2, we have a fragile UI support for SQL data writing commands. We only track the input query plan of `FileFormatWriter` and display its metrics. This is not ideal because we don't know who triggered the writing(can be table insertion, CTAS, etc.), but it's still useful to see the metrics of the input query.

In Spark 2.3, we introduced a new mechanism: `DataWritigCommand`, to fix the UI issue entirely. Now these writing commands have real children, and we don't need to hack into the `FileFormatWriter` for the UI. This also helps with `explain`, now `explain` can show the physical plan of the input query, while in 2.2 the physical writing plan is simply `ExecutedCommandExec` and it has no child.

However there is a regression in CTAS. CTAS commands don't extend `DataWritigCommand`, and we don't have the UI hack in `FileFormatWriter` anymore, so the UI for CTAS is just an empty node. See https://issues.apache.org/jira/browse/SPARK-22977 for more information about this UI issue.

To fix it, we should apply the `DataWritigCommand` mechanism to CTAS commands.

TODO: In the future, we should refactor this part and create some physical layer code pieces for data writing, and reuse them in different writing commands. We should have different logical nodes for different operators, even some of them share some same logic, e.g. CTAS, CREATE TABLE, INSERT TABLE. Internally we can share the same physical logic.

## How was this patch tested?

manually tested.
For data source table
<img width="644" alt="1" src="https://user-images.githubusercontent.com/3182036/35874155-bdffab28-0ba6-11e8-94a8-e32e106ba069.png">
For hive table
<img width="666" alt="2" src="https://user-images.githubusercontent.com/3182036/35874161-c437e2a8-0ba6-11e8-98ed-7930f01432c5.png">

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20521 from cloud-fan/UI.
2018-02-12 22:07:59 +08:00
caoxuewen caeb108e25 [MINOR][TEST] spark.testing` No effect on the SparkFunSuite unit test
## What changes were proposed in this pull request?

Currently, we use SBT and MAVN to spark unit test, are affected by the parameters of `spark.testing`. However, when using the IDE test tool, `spark.testing` support is not very good, sometimes need to be manually added to the beforeEach. example: HiveSparkSubmitSuite RPackageUtilsSuite SparkSubmitSuite. The PR unified `spark.testing` parameter extraction to SparkFunSuite, support IDE test tool, and the test code is more compact.

## How was this patch tested?

the existed test cases.

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

Closes #20582 from heary-cao/sparktesting.
2018-02-12 22:05:27 +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
Wenchen Fan 6efd5d117e [SPARK-23390][SQL] Flaky Test Suite: FileBasedDataSourceSuite in Spark 2.3/hadoop 2.7
## What changes were proposed in this pull request?

This test only fails with sbt on Hadoop 2.7, I can't reproduce it locally, but here is my speculation by looking at the code:
1. FileSystem.delete doesn't delete the directory entirely, somehow we can still open the file as a 0-length empty file.(just speculation)
2. ORC intentionally allow empty files, and the reader fails during reading without closing the file stream.

This PR improves the test to make sure all files are deleted and can't be opened.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20584 from cloud-fan/flaky-test.
2018-02-11 23:46:23 -08:00
Wenchen Fan 4bbd7443eb [SPARK-23376][SQL] creating UnsafeKVExternalSorter with BytesToBytesMap may fail
## What changes were proposed in this pull request?

This is a long-standing bug in `UnsafeKVExternalSorter` and was reported in the dev list multiple times.

When creating `UnsafeKVExternalSorter` with `BytesToBytesMap`, we need to create a `UnsafeInMemorySorter` to sort the data in `BytesToBytesMap`. The data format of the sorter and the map is same, so no data movement is required. However, both the sorter and the map need a point array for some bookkeeping work.

There is an optimization in `UnsafeKVExternalSorter`: reuse the point array between the sorter and the map, to avoid an extra memory allocation. This sounds like a reasonable optimization, the length of the `BytesToBytesMap` point array is at least 4 times larger than the number of keys(to avoid hash collision, the hash table size should be at least 2 times larger than the number of keys, and each key occupies 2 slots). `UnsafeInMemorySorter` needs the pointer array size to be 4 times of the number of entries, so we are safe to reuse the point array.

However, the number of keys of the map doesn't equal to the number of entries in the map, because `BytesToBytesMap` supports duplicated keys. This breaks the assumption of the above optimization and we may run out of space when inserting data into the sorter, and hit error
```
java.lang.IllegalStateException: There is no space for new record
   at org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.insertRecord(UnsafeInMemorySorter.java:239)
   at org.apache.spark.sql.execution.UnsafeKVExternalSorter.<init>(UnsafeKVExternalSorter.java:149)
...
```

This PR fixes this bug by creating a new point array if the existing one is not big enough.

## How was this patch tested?

a new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20561 from cloud-fan/bug.
2018-02-12 00:03:49 +08:00
Feng Liu 6d7c38330e [SPARK-23275][SQL] fix the thread leaking in hive/tests
## What changes were proposed in this pull request?

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

The two lines actually can trigger the hive metastore bug: https://issues.apache.org/jira/browse/HIVE-16844

The two configs are not in the default `ObjectStore` properties, so any run hive commands after these two lines will set the `propsChanged` flag in the `ObjectStore.setConf` and then cause thread leaks.

I don't think the two lines are very useful. They can be removed safely.

## 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: Feng Liu <fengliu@databricks.com>

Closes #20562 from liufengdb/fix-omm.
2018-02-09 16:21:47 -08:00
Jacek Laskowski 557938e283 [MINOR][HIVE] Typo fixes
## What changes were proposed in this pull request?

Typo fixes (with expanding a Hive property)

## How was this patch tested?

local build. Awaiting Jenkins

Author: Jacek Laskowski <jacek@japila.pl>

Closes #20550 from jaceklaskowski/hiveutils-typos.
2018-02-09 18:18:30 -06:00
Dongjoon Hyun 8cbcc33876 [SPARK-23186][SQL] Initialize DriverManager first before loading JDBC Drivers
## What changes were proposed in this pull request?

Since some JDBC Drivers have class initialization code to call `DriverManager`, we need to initialize `DriverManager` first in order to avoid potential executor-side **deadlock** situations like the following (or [STORM-2527](https://issues.apache.org/jira/browse/STORM-2527)).

```
Thread 9587: (state = BLOCKED)
 - sun.reflect.NativeConstructorAccessorImpl.newInstance0(java.lang.reflect.Constructor, java.lang.Object[]) bci=0 (Compiled frame; information may be imprecise)
 - sun.reflect.NativeConstructorAccessorImpl.newInstance(java.lang.Object[]) bci=85, line=62 (Compiled frame)
 - sun.reflect.DelegatingConstructorAccessorImpl.newInstance(java.lang.Object[]) bci=5, line=45 (Compiled frame)
 - java.lang.reflect.Constructor.newInstance(java.lang.Object[]) bci=79, line=423 (Compiled frame)
 - java.lang.Class.newInstance() bci=138, line=442 (Compiled frame)
 - java.util.ServiceLoader$LazyIterator.nextService() bci=119, line=380 (Interpreted frame)
 - java.util.ServiceLoader$LazyIterator.next() bci=11, line=404 (Interpreted frame)
 - java.util.ServiceLoader$1.next() bci=37, line=480 (Interpreted frame)
 - java.sql.DriverManager$2.run() bci=21, line=603 (Interpreted frame)
 - java.sql.DriverManager$2.run() bci=1, line=583 (Interpreted frame)
 - java.security.AccessController.doPrivileged(java.security.PrivilegedAction) bci=0 (Compiled frame)
 - java.sql.DriverManager.loadInitialDrivers() bci=27, line=583 (Interpreted frame)
 - java.sql.DriverManager.<clinit>() bci=32, line=101 (Interpreted frame)
 - org.apache.phoenix.mapreduce.util.ConnectionUtil.getConnection(java.lang.String, java.lang.Integer, java.lang.String, java.util.Properties) bci=12, line=98 (Interpreted frame)
 - org.apache.phoenix.mapreduce.util.ConnectionUtil.getInputConnection(org.apache.hadoop.conf.Configuration, java.util.Properties) bci=22, line=57 (Interpreted frame)
 - org.apache.phoenix.mapreduce.PhoenixInputFormat.getQueryPlan(org.apache.hadoop.mapreduce.JobContext, org.apache.hadoop.conf.Configuration) bci=61, line=116 (Interpreted frame)
 - org.apache.phoenix.mapreduce.PhoenixInputFormat.createRecordReader(org.apache.hadoop.mapreduce.InputSplit, org.apache.hadoop.mapreduce.TaskAttemptContext) bci=10, line=71 (Interpreted frame)
 - org.apache.spark.rdd.NewHadoopRDD$$anon$1.<init>(org.apache.spark.rdd.NewHadoopRDD, org.apache.spark.Partition, org.apache.spark.TaskContext) bci=233, line=156 (Interpreted frame)

Thread 9170: (state = BLOCKED)
 - org.apache.phoenix.jdbc.PhoenixDriver.<clinit>() bci=35, line=125 (Interpreted frame)
 - sun.reflect.NativeConstructorAccessorImpl.newInstance0(java.lang.reflect.Constructor, java.lang.Object[]) bci=0 (Compiled frame)
 - sun.reflect.NativeConstructorAccessorImpl.newInstance(java.lang.Object[]) bci=85, line=62 (Compiled frame)
 - sun.reflect.DelegatingConstructorAccessorImpl.newInstance(java.lang.Object[]) bci=5, line=45 (Compiled frame)
 - java.lang.reflect.Constructor.newInstance(java.lang.Object[]) bci=79, line=423 (Compiled frame)
 - java.lang.Class.newInstance() bci=138, line=442 (Compiled frame)
 - org.apache.spark.sql.execution.datasources.jdbc.DriverRegistry$.register(java.lang.String) bci=89, line=46 (Interpreted frame)
 - org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$$anonfun$createConnectionFactory$2.apply() bci=7, line=53 (Interpreted frame)
 - org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$$anonfun$createConnectionFactory$2.apply() bci=1, line=52 (Interpreted frame)
 - org.apache.spark.sql.execution.datasources.jdbc.JDBCRDD$$anon$1.<init>(org.apache.spark.sql.execution.datasources.jdbc.JDBCRDD, org.apache.spark.Partition, org.apache.spark.TaskContext) bci=81, line=347 (Interpreted frame)
 - org.apache.spark.sql.execution.datasources.jdbc.JDBCRDD.compute(org.apache.spark.Partition, org.apache.spark.TaskContext) bci=7, line=339 (Interpreted frame)
```

## How was this patch tested?

N/A

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20359 from dongjoon-hyun/SPARK-23186.
2018-02-09 12:54:57 +08:00
Wenchen Fan a75f927173 [SPARK-23268][SQL][FOLLOWUP] Reorganize packages in data source V2
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/20435.

While reorganizing the packages for streaming data source v2, the top level stream read/write support interfaces should not be in the reader/writer package, but should be in the `sources.v2` package, to follow the `ReadSupport`, `WriteSupport`, etc.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20509 from cloud-fan/followup.
2018-02-08 19:20:11 +08:00
Wenchen Fan 7f5f5fb129 [SPARK-23348][SQL] append data using saveAsTable should adjust the data types
## What changes were proposed in this pull request?

For inserting/appending data to an existing table, Spark should adjust the data types of the input query according to the table schema, or fail fast if it's uncastable.

There are several ways to insert/append data: SQL API, `DataFrameWriter.insertInto`, `DataFrameWriter.saveAsTable`. The first 2 ways create `InsertIntoTable` plan, and the last way creates `CreateTable` plan. However, we only adjust input query data types for `InsertIntoTable`, and users may hit weird errors when appending data using `saveAsTable`. See the JIRA for the error case.

This PR fixes this bug by adjusting data types for `CreateTable` too.

## How was this patch tested?

new test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20527 from cloud-fan/saveAsTable.
2018-02-08 00:08:54 -08:00
gatorsmile 3473fda6dc Revert [SPARK-22279][SQL] Turn on spark.sql.hive.convertMetastoreOrc by default
## What changes were proposed in this pull request?

This is to revert the changes made in https://github.com/apache/spark/pull/19499 , because this causes a regression. We should not ignore the table-specific compression conf when the Hive serde tables are converted to the data source tables.

## How was this patch tested?

The existing tests.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20536 from gatorsmile/revert22279.
2018-02-08 12:21:18 +08:00
Tathagata Das 30295bf5a6 [SPARK-23092][SQL] Migrate MemoryStream to DataSourceV2 APIs
## What changes were proposed in this pull request?

This PR migrates the MemoryStream to DataSourceV2 APIs.

One additional change is in the reported keys in StreamingQueryProgress.durationMs. "getOffset" and "getBatch" replaced with "setOffsetRange" and "getEndOffset" as tracking these make more sense. Unit tests changed accordingly.

## How was this patch tested?
Existing unit tests, few updated unit tests.

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

Closes #20445 from tdas/SPARK-23092.
2018-02-07 15:22:53 -08:00
Liang-Chi Hsieh 9841ae0313 [SPARK-23345][SQL] Remove open stream record even closing it fails
## What changes were proposed in this pull request?

When `DebugFilesystem` closes opened stream, if any exception occurs, we still need to remove the open stream record from `DebugFilesystem`. Otherwise, it goes to report leaked filesystem connection.

## How was this patch tested?

Existing tests.

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

Closes #20524 from viirya/SPARK-23345.
2018-02-07 09:48:49 -08:00
gatorsmile 9775df67f9 [SPARK-23122][PYSPARK][FOLLOWUP] Replace registerTempTable by createOrReplaceTempView
## What changes were proposed in this pull request?
Replace `registerTempTable` by `createOrReplaceTempView`.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20523 from gatorsmile/updateExamples.
2018-02-07 23:24:16 +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
Wenchen Fan b96a083b1c [SPARK-23315][SQL] failed to get output from canonicalized data source v2 related plans
## What changes were proposed in this pull request?

`DataSourceV2Relation`  keeps a `fullOutput` and resolves the real output on demand by column name lookup. i.e.
```
lazy val output: Seq[Attribute] = reader.readSchema().map(_.name).map { name =>
  fullOutput.find(_.name == name).get
}
```

This will be broken after we canonicalize the plan, because all attribute names become "None", see https://github.com/apache/spark/blob/v2.3.0-rc1/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala#L42

To fix this, `DataSourceV2Relation` should just keep `output`, and update the `output` when doing column pruning.

## How was this patch tested?

a new test case

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20485 from cloud-fan/canonicalize.
2018-02-06 12:43:45 -08:00
Wenchen Fan ac7454cac0 [SPARK-23312][SQL][FOLLOWUP] add a config to turn off vectorized cache reader
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/20483 tried to provide a way to turn off the new columnar cache reader, to restore the behavior in 2.2. However even we turn off that config, the behavior is still different than 2.2.

If the output data are rows, we still enable whole stage codegen for the scan node, which is different with 2.2, we should also fix it.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20513 from cloud-fan/cache.
2018-02-06 12:27:37 -08:00
Xingbo Jiang c2766b07b4 [SPARK-23330][WEBUI] Spark UI SQL executions page throws NPE
## What changes were proposed in this pull request?

Spark SQL executions page throws the following error and the page crashes:
```
HTTP ERROR 500
Problem accessing /SQL/. Reason:

Server Error
Caused by:
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.execution.ui.ExecutionTable.descriptionCell(AllExecutionsPage.scala:182)
at org.apache.spark.sql.execution.ui.ExecutionTable.row(AllExecutionsPage.scala:155)
at org.apache.spark.sql.execution.ui.ExecutionTable$$anonfun$8.apply(AllExecutionsPage.scala:204)
at org.apache.spark.sql.execution.ui.ExecutionTable$$anonfun$8.apply(AllExecutionsPage.scala:204)
at org.apache.spark.ui.UIUtils$$anonfun$listingTable$2.apply(UIUtils.scala:339)
at org.apache.spark.ui.UIUtils$$anonfun$listingTable$2.apply(UIUtils.scala:339)
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.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
at scala.collection.AbstractTraversable.map(Traversable.scala:104)
at org.apache.spark.ui.UIUtils$.listingTable(UIUtils.scala:339)
at org.apache.spark.sql.execution.ui.ExecutionTable.toNodeSeq(AllExecutionsPage.scala:203)
at org.apache.spark.sql.execution.ui.AllExecutionsPage.render(AllExecutionsPage.scala:67)
at org.apache.spark.ui.WebUI$$anonfun$2.apply(WebUI.scala:82)
at org.apache.spark.ui.WebUI$$anonfun$2.apply(WebUI.scala:82)
at org.apache.spark.ui.JettyUtils$$anon$3.doGet(JettyUtils.scala:90)
at javax.servlet.http.HttpServlet.service(HttpServlet.java:687)
at javax.servlet.http.HttpServlet.service(HttpServlet.java:790)
at org.eclipse.jetty.servlet.ServletHolder.handle(ServletHolder.java:848)
at org.eclipse.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:584)
at org.eclipse.jetty.server.handler.ContextHandler.doHandle(ContextHandler.java:1180)
at org.eclipse.jetty.servlet.ServletHandler.doScope(ServletHandler.java:512)
at org.eclipse.jetty.server.handler.ContextHandler.doScope(ContextHandler.java:1112)
at org.eclipse.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:141)
at org.eclipse.jetty.server.handler.ContextHandlerCollection.handle(ContextHandlerCollection.java:213)
at org.eclipse.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:134)
at org.eclipse.jetty.server.Server.handle(Server.java:534)
at org.eclipse.jetty.server.HttpChannel.handle(HttpChannel.java:320)
at org.eclipse.jetty.server.HttpConnection.onFillable(HttpConnection.java:251)
at org.eclipse.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:283)
at org.eclipse.jetty.io.FillInterest.fillable(FillInterest.java:108)
at org.eclipse.jetty.io.SelectChannelEndPoint$2.run(SelectChannelEndPoint.java:93)
at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.executeProduceConsume(ExecuteProduceConsume.java:303)
at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.produceConsume(ExecuteProduceConsume.java:148)
at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.run(ExecuteProduceConsume.java:136)
at org.eclipse.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:671)
at org.eclipse.jetty.util.thread.QueuedThreadPool$2.run(QueuedThreadPool.java:589)
at java.lang.Thread.run(Thread.java:748)
```

One of the possible reason that this page fails may be the `SparkListenerSQLExecutionStart` event get dropped before processed, so the execution description and details don't get updated.
This was not a issue in 2.2 because it would ignore any job start event that arrives before the corresponding execution start event, which doesn't sound like a good decision.

We shall try to handle the null values in the front page side, that is, try to give a default value when `execution.details` or `execution.description` is null.
Another possible approach is not to spill the `LiveExecutionData` in `SQLAppStatusListener.update(exec: LiveExecutionData)` if `exec.details` is null. This is not ideal because this way you will not see the execution if `SparkListenerSQLExecutionStart` event is lost, because `AllExecutionsPage` only read executions from KVStore.

## How was this patch tested?

After the change, the page shows the following:
![image](https://user-images.githubusercontent.com/4784782/35775480-28cc5fde-093e-11e8-8ccc-f58c2ef4a514.png)

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

Closes #20502 from jiangxb1987/executionPage.
2018-02-05 14:17:11 -08:00
Shixiong Zhu a6bf3db207 [SPARK-23307][WEBUI] Sort jobs/stages/tasks/queries with the completed timestamp before cleaning up them
## What changes were proposed in this pull request?

Sort jobs/stages/tasks/queries with the completed timestamp before cleaning up them to make the behavior consistent with 2.2.

## How was this patch tested?

- Jenkins.
- Manually ran the following codes and checked the UI for jobs/stages/tasks/queries.

```
spark.ui.retainedJobs 10
spark.ui.retainedStages 10
spark.sql.ui.retainedExecutions 10
spark.ui.retainedTasks 10
```

```
new Thread() {
  override def run() {
    spark.range(1, 2).foreach { i =>
        Thread.sleep(10000)
    }
  }
}.start()

Thread.sleep(5000)

for (_ <- 1 to 20) {
    new Thread() {
      override def run() {
        spark.range(1, 2).foreach { i =>
        }
      }
    }.start()
}

Thread.sleep(15000)
  spark.range(1, 2).foreach { i =>
}

sc.makeRDD(1 to 100, 100).foreach { i =>
}
```

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #20481 from zsxwing/SPARK-23307.
2018-02-05 18:41:49 +08:00
Yuming Wang 6fb3fd1536 [SPARK-22036][SQL][FOLLOWUP] Fix decimalArithmeticOperations.sql
## What changes were proposed in this pull request?

Fix decimalArithmeticOperations.sql test

## How was this patch tested?

N/A

Author: Yuming Wang <wgyumg@gmail.com>
Author: wangyum <wgyumg@gmail.com>
Author: Yuming Wang <yumwang@ebay.com>

Closes #20498 from wangyum/SPARK-22036.
2018-02-04 09:15:48 -08:00
Dongjoon Hyun 522e0b1866 [SPARK-23305][SQL][TEST] Test spark.sql.files.ignoreMissingFiles for all file-based data sources
## What changes were proposed in this pull request?

Like Parquet, all file-based data source handles `spark.sql.files.ignoreMissingFiles` correctly. We had better have a test coverage for feature parity and in order to prevent future accidental regression for all data sources.

## How was this patch tested?

Pass Jenkins with a newly added test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20479 from dongjoon-hyun/SPARK-23305.
2018-02-03 00:04:00 -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 fe73cb4b43 [SPARK-23317][SQL] rename ContinuousReader.setOffset to setStartOffset
## What changes were proposed in this pull request?

In the document of `ContinuousReader.setOffset`, we say this method is used to specify the start offset. We also have a `ContinuousReader.getStartOffset` to get the value back. I think it makes more sense to rename `ContinuousReader.setOffset` to `setStartOffset`.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20486 from cloud-fan/rename.
2018-02-02 20:49:08 -08:00
Reynold Xin 3ff83ad43a [SQL] Minor doc update: Add an example in DataFrameReader.schema
## What changes were proposed in this pull request?
This patch adds a small example to the schema string definition of schema function. It isn't obvious how to use it, so an example would be useful.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #20491 from rxin/schema-doc.
2018-02-02 20:36:27 -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
Wenchen Fan 19c7c7ebde [SPARK-23301][SQL] data source column pruning should work for arbitrary expressions
## What changes were proposed in this pull request?

This PR fixes a mistake in the `PushDownOperatorsToDataSource` rule, the column pruning logic is incorrect about `Project`.

## How was this patch tested?

a new test case for column pruning with arbitrary expressions, and improve the existing tests to make sure the `PushDownOperatorsToDataSource` really works.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20476 from cloud-fan/push-down.
2018-02-01 20:44:46 -08:00
Liang-Chi Hsieh 90848d5074 [SPARK-23284][SQL] Document the behavior of several ColumnVector's get APIs when accessing null slot
## What changes were proposed in this pull request?

For some ColumnVector get APIs such as getDecimal, getBinary, getStruct, getArray, getInterval, getUTF8String, we should clearly document their behaviors when accessing null slot. They should return null in this case. Then we can remove null checks from the places using above APIs.

For the APIs of primitive values like getInt, getInts, etc., this also documents their behaviors when accessing null slots. Their returning values are undefined and can be anything.

## How was this patch tested?

Added tests into `ColumnarBatchSuite`.

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

Closes #20455 from viirya/SPARK-23272-followup.
2018-02-02 10:18:32 +08:00
Wenchen Fan 73da3b6968 [SPARK-23293][SQL] fix data source v2 self join
## What changes were proposed in this pull request?

`DataSourceV2Relation` should extend `MultiInstanceRelation`, to take care of self-join.

## How was this patch tested?

a new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20466 from cloud-fan/dsv2-selfjoin.
2018-02-01 10:48:34 -08:00
Yuming Wang f051f83403 [SPARK-13983][SQL] Fix HiveThriftServer2 can not get "--hiveconf" and ''--hivevar" variables since 2.0
## What changes were proposed in this pull request?

`--hiveconf` and `--hivevar` variables no longer work since Spark 2.0. The `spark-sql` client has fixed by [SPARK-15730](https://issues.apache.org/jira/browse/SPARK-15730) and [SPARK-18086](https://issues.apache.org/jira/browse/SPARK-18086). but `beeline`/[`Spark SQL HiveThriftServer2`](https://github.com/apache/spark/blob/v2.1.1/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala) is still broken. This pull request fix it.

This pull request works for both `JDBC client` and `beeline`.

## How was this patch tested?

unit tests for  `JDBC client`
manual tests for `beeline`:
```
git checkout origin/pr/17886

dev/make-distribution.sh --mvn mvn  --tgz -Phive -Phive-thriftserver -Phadoop-2.6 -DskipTests

tar -zxf spark-2.3.0-SNAPSHOT-bin-2.6.5.tgz && cd spark-2.3.0-SNAPSHOT-bin-2.6.5

sbin/start-thriftserver.sh
```
```
cat <<EOF > test.sql
select '\${a}', '\${b}';
EOF

beeline -u jdbc:hive2://localhost:10000 --hiveconf a=avalue --hivevar b=bvalue -f test.sql

```

Author: Yuming Wang <wgyumg@gmail.com>

Closes #17886 from wangyum/SPARK-13983-dev.
2018-02-01 10:36:31 -08:00
Wang Gengliang ffbca84519 [SPARK-23202][SQL] Add new API in DataSourceWriter: onDataWriterCommit
## What changes were proposed in this pull request?

The current DataSourceWriter API makes it hard to implement `onTaskCommit(taskCommit: TaskCommitMessage)` in `FileCommitProtocol`.
In general, on receiving commit message, driver can start processing messages(e.g. persist messages into files) before all the messages are collected.

The proposal to add a new API:
`add(WriterCommitMessage message)`:  Handles a commit message on receiving from a successful data writer.

This should make the whole API of DataSourceWriter compatible with `FileCommitProtocol`, and more flexible.

There was another radical attempt in #20386.  This one should be more reasonable.

## How was this patch tested?

Unit test

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #20454 from gengliangwang/write_api.
2018-02-01 20:39:15 +08:00
Takuya UESHIN 89e8d556b9 [SPARK-23280][SQL][FOLLOWUP] Enable MutableColumnarRow.getMap().
## What changes were proposed in this pull request?

This is a followup pr of #20450.
We should've enabled `MutableColumnarRow.getMap()` as well.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #20471 from ueshin/issues/SPARK-23280/fup2.
2018-02-01 21:28:53 +09:00
Takuya UESHIN 8bb70b068e [SPARK-23280][SQL][FOLLOWUP] Fix Java style check issues.
## What changes were proposed in this pull request?

This is a follow-up of #20450 which broke lint-java checks.
This pr fixes the lint-java issues.

```
[ERROR] src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java:[20,8] (imports) UnusedImports: Unused import - org.apache.spark.sql.catalyst.util.MapData.
[ERROR] src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java:[21,8] (imports) UnusedImports: Unused import - org.apache.spark.sql.catalyst.util.MapData.
[ERROR] src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java:[22,8] (imports) UnusedImports: Unused import - org.apache.spark.sql.catalyst.util.MapData.
```

## How was this patch tested?

Checked manually in my local environment.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #20468 from ueshin/issues/SPARK-23280/fup1.
2018-02-01 21:25:02 +09: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
Atallah Hezbor b2e7677f4d [SPARK-21396][SQL] Fixes MatchError when UDTs are passed through Hive Thriftserver
Signed-off-by: Atallah Hezbor <atallahhezborgmail.com>

## What changes were proposed in this pull request?

This PR proposes modifying the match statement that gets the columns of a row in HiveThriftServer. There was previously no case for `UserDefinedType`, so querying a table that contained them would throw a match error. The changes catch that case and return the string representation.

## How was this patch tested?

While I would have liked to add a unit test, I couldn't easily incorporate UDTs into the ``HiveThriftServer2Suites`` pipeline. With some guidance I would be happy to push a commit with tests.

Instead I did a manual test by loading a `DataFrame` with Point UDT in a spark shell with a HiveThriftServer. Then in beeline, connecting to the server and querying that table.

Here is the result before the change
```
0: jdbc:hive2://localhost:10000> select * from chicago;
Error: scala.MatchError: org.apache.spark.sql.PointUDT2d980dc3 (of class org.apache.spark.sql.PointUDT) (state=,code=0)

```

And after the change:
```
0: jdbc:hive2://localhost:10000> select * from chicago;
+---------------------------------------+--------------+------------------------+---------------------+--+
|                __fid__                | case_number  |          dtg           |        geom         |
+---------------------------------------+--------------+------------------------+---------------------+--+
| 109602f9-54f8-414b-8c6f-42b1a337643e  | 2            | 2016-01-01 19:00:00.0  | POINT (-77 38)      |
| 709602f9-fcff-4429-8027-55649b6fd7ed  | 1            | 2015-12-31 19:00:00.0  | POINT (-76.5 38.5)  |
| 009602f9-fcb5-45b1-a867-eb8ba10cab40  | 3            | 2016-01-02 19:00:00.0  | POINT (-78 39)      |
+---------------------------------------+--------------+------------------------+---------------------+--+
```

Author: Atallah Hezbor <atallahhezbor@gmail.com>

Closes #20385 from atallahhezbor/udts_over_hive.
2018-01-31 20:45:55 -08:00
Wang Gengliang 56ae32657e [SPARK-23268][SQL] Reorganize packages in data source V2
## What changes were proposed in this pull request?
1. create a new package for partitioning/distribution related classes.
    As Spark will add new concrete implementations of `Distribution` in new releases, it is good to
    have a new package for partitioning/distribution related classes.

2. move streaming related class to package `org.apache.spark.sql.sources.v2.reader/writer.streaming`, instead of `org.apache.spark.sql.sources.v2.streaming.reader/writer`.
So that the there won't be package reader/writer inside package streaming, which is quite confusing.
Before change:
```
v2
├── reader
├── streaming
│   ├── reader
│   └── writer
└── writer
```

After change:
```
v2
├── reader
│   └── streaming
└── writer
    └── streaming
```
## How was this patch tested?
Unit test.

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #20435 from gengliangwang/new_pkg.
2018-01-31 20:33:51 -08:00
caoxuewen 2ac895be90 [SPARK-23247][SQL] combines Unsafe operations and statistics operations in Scan Data Source
## What changes were proposed in this pull request?

Currently, we scan the execution plan of the data source, first the unsafe operation of each row of data, and then re traverse the data for the count of rows. In terms of performance, this is not necessary. this PR combines the two operations and makes statistics on the number of rows while performing the unsafe operation.

Before modified,

```
val unsafeRow = rdd.mapPartitionsWithIndexInternal { (index, iter) =>
val proj = UnsafeProjection.create(schema)
 proj.initialize(index)
iter.map(proj)
}

val numOutputRows = longMetric("numOutputRows")
unsafeRow.map { r =>
numOutputRows += 1
 r
}
```
After modified,

    val numOutputRows = longMetric("numOutputRows")

    rdd.mapPartitionsWithIndexInternal { (index, iter) =>
      val proj = UnsafeProjection.create(schema)
      proj.initialize(index)
      iter.map( r => {
        numOutputRows += 1
        proj(r)
      })
    }

## How was this patch tested?

the existed test cases.

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

Closes #20415 from heary-cao/DataSourceScanExec.
2018-02-01 12:05:12 +08:00
Wenchen Fan 52e00f7066 [SPARK-23280][SQL] add map type support to ColumnVector
## What changes were proposed in this pull request?

Fill the last missing piece of `ColumnVector`: the map type support.

The idea is similar to the array type support. A map is basically 2 arrays: keys and values. We ask the implementations to provide a key array, a value array, and an offset and length to specify the range of this map in the key/value array.

In `WritableColumnVector`, we put the key array in first child vector, and value array in second child vector, and offsets and lengths in the current vector, which is very similar to how array type is implemented here.

## How was this patch tested?

a new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20450 from cloud-fan/map.
2018-02-01 11:56:06 +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
Glen Takahashi 8c21170dec [SPARK-23249][SQL] Improved block merging logic for partitions
## What changes were proposed in this pull request?

Change DataSourceScanExec so that when grouping blocks together into partitions, also checks the end of the sorted list of splits to more efficiently fill out partitions.

## How was this patch tested?

Updated old test to reflect the new logic, which causes the # of partitions to drop from 4 -> 3
Also, a current test exists to test large non-splittable files at c575977a59/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala (L346)

## Rationale

The current bin-packing method of next-fit descending for blocks into partitions is sub-optimal in a lot of cases and will result in extra partitions, un-even distribution of block-counts across partitions, and un-even distribution of partition sizes.

As an example, 128 files ranging from 1MB, 2MB,...127MB,128MB. will result in 82 partitions with the current algorithm, but only 64 using this algorithm. Also in this example, the max # of blocks per partition in NFD is 13, while in this algorithm is is 2.

More generally, running a simulation of 1000 runs using 128MB blocksize, between 1-1000 normally distributed file sizes between 1-500Mb, you can see an improvement of approx 5% reduction of partition counts, and a large reduction in standard deviation of blocks per partition.

This algorithm also runs in O(n) time as NFD does, and in every case is strictly better results than NFD.

Overall, the more even distribution of blocks across partitions and therefore reduced partition counts should result in a small but significant performance increase across the board

Author: Glen Takahashi <gtakahashi@palantir.com>

Closes #20372 from glentakahashi/feature/improved-block-merging.
2018-02-01 01:14:01 +08:00
Wenchen Fan 48dd6a4c79 revert [SPARK-22785][SQL] remove ColumnVector.anyNullsSet
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/19980 , we thought `anyNullsSet` can be simply implemented by `numNulls() > 0`. This is logically true, but may have performance problems.

`OrcColumnVector` is an example. It doesn't have the `numNulls` property, only has a `noNulls` property. We will lose a lot of performance if we use `numNulls() > 0` to check null.

This PR simply revert #19980, with a renaming to call it `hasNull`. Better name suggestions are welcome, e.g. `nullable`?

## How was this patch tested?

existing test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20452 from cloud-fan/null.
2018-02-01 00:24:42 +08:00
Wenchen Fan 695f7146bc [SPARK-23272][SQL] add calendar interval type support to ColumnVector
## What changes were proposed in this pull request?

`ColumnVector` is aimed to support all the data types, but `CalendarIntervalType` is missing. Actually we do support interval type for inner fields, e.g. `ColumnarRow`, `ColumnarArray` both support interval type. It's weird if we don't support interval type at the top level.

This PR adds the interval type support.

This PR also makes `ColumnVector.getChild` protect. We need it public because `MutableColumnaRow.getInterval` needs it. Now the interval implementation is in `ColumnVector.getInterval`.

## How was this patch tested?

a new test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20438 from cloud-fan/interval.
2018-01-31 15:13:15 +08:00
jerryshao 8c6a9c90a3 [SPARK-23279][SS] Avoid triggering distributed job for Console sink
## What changes were proposed in this pull request?

Console sink will redistribute collected local data and trigger a distributed job in each batch, this is not necessary, so here change to local job.

## How was this patch tested?

Existing UT and manual verification.

Author: jerryshao <sshao@hortonworks.com>

Closes #20447 from jerryshao/console-minor.
2018-01-31 13:59:21 +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
Dongjoon Hyun 7786616733 [SPARK-23276][SQL][TEST] Enable UDT tests in (Hive)OrcHadoopFsRelationSuite
## What changes were proposed in this pull request?

Like Parquet, ORC test suites should enable UDT tests.

## How was this patch tested?

Pass the Jenkins with newly enabled test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20440 from dongjoon-hyun/SPARK-23276.
2018-01-30 17:14:17 -08:00
Dilip Biswal 58fcb5a95e [SPARK-23275][SQL] hive/tests have been failing when run locally on the laptop (Mac) with OOM
## What changes were proposed in this pull request?
hive tests have been failing when they are run locally (Mac Os) after a recent change in the trunk. After running the tests for some time, the test fails with OOM with Error: unable to create new native thread.

I noticed the thread count goes all the way up to 2000+ after which we start getting these OOM errors. Most of the threads seem to be related to the connection pool in hive metastore (BoneCP-xxxxx-xxxx ). This behaviour change is happening after we made the following change to HiveClientImpl.reset()

``` SQL
 def reset(): Unit = withHiveState {
    try {
      // code
    } finally {
      runSqlHive("USE default")  ===> this is causing the issue
    }
```
I am proposing to temporarily back-out part of a fix made to address SPARK-23000 to resolve this issue while we work-out the exact reason for this sudden increase in thread counts.

## How was this patch tested?
Ran hive/test multiple times in different machines.

(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: Dilip Biswal <dbiswal@us.ibm.com>

Closes #20441 from dilipbiswal/hive_tests.
2018-01-30 14:11:06 -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
Liang-Chi Hsieh 84bcf9dc88 [SPARK-23222][SQL] Make DataFrameRangeSuite not flaky
## What changes were proposed in this pull request?

It is reported that the test `Cancelling stage in a query with Range` in `DataFrameRangeSuite` fails a few times in unrelated PRs. I personally also saw it too in my PR.

This test is not very flaky actually but only fails occasionally. Based on how the test works, I guess that is because `range` finishes before the listener calls `cancelStage`.

I increase the range number from `1000000000L` to `100000000000L` and count the range in one partition. I also reduce the `interval` of checking stage id. Hopefully it can make the test not flaky anymore.

## How was this patch tested?

The modified tests.

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

Closes #20431 from viirya/SPARK-23222.
2018-01-30 21:00:29 +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
Wenchen Fan 0a9ac0248b [SPARK-23260][SPARK-23262][SQL] several data source v2 naming cleanup
## What changes were proposed in this pull request?

All other classes in the reader/writer package doesn't have `V2` in their names, and the streaming reader/writer don't have `V2` either. It's more consistent to remove `V2` from `DataSourceV2Reader` and `DataSourceVWriter`.

Also rename `DataSourceV2Option` to remote the `V2`, we should only have `V2` in the root interface: `DataSourceV2`.

This PR also fixes some places that the mix-in interface doesn't extend the interface it aimed to mix in.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20427 from cloud-fan/ds-v2.
2018-01-30 19:43:17 +08:00
Henry Robinson 8b983243e4 [SPARK-23157][SQL] Explain restriction on column expression in withColumn()
## What changes were proposed in this pull request?

It's not obvious from the comments that any added column must be a
function of the dataset that we are adding it to. Add a comment to
that effect to Scala, Python and R Data* methods.

Author: Henry Robinson <henry@cloudera.com>

Closes #20429 from henryr/SPARK-23157.
2018-01-29 22:19:59 -08:00
Xingbo Jiang b375397b16 [SPARK-23207][SQL][FOLLOW-UP] Don't perform local sort for DataFrame.repartition(1)
## What changes were proposed in this pull request?

In `ShuffleExchangeExec`, we don't need to insert extra local sort before round-robin partitioning, if the new partitioning has only 1 partition, because under that case all output rows go to the same partition.

## How was this patch tested?

The existing test cases.

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

Closes #20426 from jiangxb1987/repartition1.
2018-01-30 11:40:42 +08: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
gatorsmile e30b34f7bd [SPARK-22916][SQL][FOLLOW-UP] Update the Description of Join Selection
## What changes were proposed in this pull request?
This PR is to update the description of the join algorithm changes.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20420 from gatorsmile/followUp22916.
2018-01-29 10:29:42 -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
xubo245 fbce2ed0fa [SPARK-23059][SQL][TEST] Correct some improper with view related method usage
## What changes were proposed in this pull request?

Correct some improper with view related method usage
Only change test cases

like:

```
 test("list global temp views") {
    try {
      sql("CREATE GLOBAL TEMP VIEW v1 AS SELECT 3, 4")
      sql("CREATE TEMP VIEW v2 AS SELECT 1, 2")

      checkAnswer(sql(s"SHOW TABLES IN $globalTempDB"),
        Row(globalTempDB, "v1", true) ::
        Row("", "v2", true) :: Nil)

      assert(spark.catalog.listTables(globalTempDB).collect().toSeq.map(_.name) == Seq("v1", "v2"))
    } finally {
      spark.catalog.dropTempView("v1")
      spark.catalog.dropGlobalTempView("v2")
    }
  }
```

other change please review the code.
## How was this patch tested?

See test case.

Author: xubo245 <601450868@qq.com>

Closes #20250 from xubo245/DropTempViewError.
2018-01-29 08:58:14 -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
CCInCharge 686a622c93 [SPARK-23250][DOCS] Typo in JavaDoc/ScalaDoc for DataFrameWriter
## What changes were proposed in this pull request?

Fix typo in ScalaDoc for DataFrameWriter - originally stated "This is applicable for all file-based data sources (e.g. Parquet, JSON) staring Spark 2.1.0", should be "starting with Spark 2.1.0".

## How was this patch tested?

Check of correct spelling in ScalaDoc

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

Author: CCInCharge <charles.l.chen.clc@gmail.com>

Closes #20417 from CCInCharge/master.
2018-01-28 14:55:43 -06:00
Jose Torres 6328868e52 [SPARK-23245][SS][TESTS] Don't access lastExecution.executedPlan in StreamTest
## What changes were proposed in this pull request?

`lastExecution.executedPlan` is lazy val so accessing it in StreamTest may need to acquire the lock of `lastExecution`. It may be waiting forever when the streaming thread is holding it and running a continuous Spark job.

This PR changes to check if `s.lastExecution` is null to avoid accessing `lastExecution.executedPlan`.

## How was this patch tested?

Jenkins

Author: Jose Torres <jose@databricks.com>

Closes #20413 from zsxwing/SPARK-23245.
2018-01-26 23:06:03 -08:00
Dongjoon Hyun e7bc9f0524 [MINOR][SS][DOC] Fix Trigger Scala/Java doc examples
## What changes were proposed in this pull request?

This PR fixes Scala/Java doc examples in `Trigger.java`.

## How was this patch tested?

N/A.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20401 from dongjoon-hyun/SPARK-TRIGGER.
2018-01-26 18:57:32 -06:00
Wenchen Fan 5b5447c68a [SPARK-23214][SQL] cached data should not carry extra hint info
## What changes were proposed in this pull request?

This is a regression introduced by https://github.com/apache/spark/pull/19864

When we lookup cache, we should not carry the hint info, as this cache entry might be added by a plan having hint info, while the input plan for this lookup may not have hint info, or have different hint info.

## How was this patch tested?

a new test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20394 from cloud-fan/cache.
2018-01-26 16:46:51 -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
Wenchen Fan dd8e257d1c [SPARK-23218][SQL] simplify ColumnVector.getArray
## What changes were proposed in this pull request?

`ColumnVector` is very flexible about how to implement array type. As a result `ColumnVector` has 3 abstract methods for array type: `arrayData`, `getArrayOffset`, `getArrayLength`. For example, in `WritableColumnVector` we use the first child vector as the array data vector, and store offsets and lengths in 2 arrays in the parent vector. `ArrowColumnVector` has a different implementation.

This PR simplifies `ColumnVector` by using only one abstract method for array type: `getArray`.

## How was this patch tested?

existing tests.

rerun `ColumnarBatchBenchmark`, there is no performance regression.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20395 from cloud-fan/vector.
2018-01-26 09:17:05 -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
Huaxin Gao 8480c0c576 [SPARK-23081][PYTHON] Add colRegex API to PySpark
## What changes were proposed in this pull request?

Add colRegex API to PySpark

## How was this patch tested?

add a test in sql/tests.py

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

Closes #20390 from huaxingao/spark-23081.
2018-01-26 07:50:48 +09: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