Commit graph

4164 commits

Author SHA1 Message Date
gatorsmile 530fe68329 [SPARK-21904][SQL] Rename tempTables to tempViews in SessionCatalog
### What changes were proposed in this pull request?
`tempTables` is not right. To be consistent, we need to rename the internal variable names/comments to tempViews in SessionCatalog too.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19117 from gatorsmile/renameTempTablesToTempViews.
2017-09-29 19:35:32 -07:00
gatorsmile 9ed7394a68 [SPARK-22161][SQL] Add Impala-modified TPC-DS queries
## What changes were proposed in this pull request?

Added IMPALA-modified TPCDS queries to TPC-DS query suites.

- Ref: https://github.com/cloudera/impala-tpcds-kit/tree/master/queries

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19386 from gatorsmile/addImpalaQueries.
2017-09-29 08:59:42 -07:00
Reynold Xin 323806e68f [SPARK-22160][SQL] Make sample points per partition (in range partitioner) configurable and bump the default value up to 100
## What changes were proposed in this pull request?
Spark's RangePartitioner hard codes the number of sampling points per partition to be 20. This is sometimes too low. This ticket makes it configurable, via spark.sql.execution.rangeExchange.sampleSizePerPartition, and raises the default in Spark SQL to be 100.

## How was this patch tested?
Added a pretty sophisticated test based on chi square test ...

Author: Reynold Xin <rxin@databricks.com>

Closes #19387 from rxin/SPARK-22160.
2017-09-28 21:07:12 -07:00
Reynold Xin d74dee1336 [SPARK-22153][SQL] Rename ShuffleExchange -> ShuffleExchangeExec
## What changes were proposed in this pull request?
For some reason when we added the Exec suffix to all physical operators, we missed this one. I was looking for this physical operator today and couldn't find it, because I was looking for ExchangeExec.

## How was this patch tested?
This is a simple rename and should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #19376 from rxin/SPARK-22153.
2017-09-28 09:20:37 -07:00
gatorsmile 9244957b50 [SPARK-22140] Add TPCDSQuerySuite
## What changes were proposed in this pull request?
Now, we are not running TPC-DS queries as regular test cases. Thus, we need to add a test suite using empty tables for ensuring the new code changes will not break them. For example, optimizer/analyzer batches should not exceed the max iteration.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19361 from gatorsmile/tpcdsQuerySuite.
2017-09-27 17:03:42 -07:00
Herman van Hovell 02bb0682e6 [SPARK-22143][SQL] Fix memory leak in OffHeapColumnVector
## What changes were proposed in this pull request?
`WriteableColumnVector` does not close its child column vectors. This can create memory leaks for `OffHeapColumnVector` where we do not clean up the memory allocated by a vectors children. This can be especially bad for string columns (which uses a child byte column vector).

## How was this patch tested?
I have updated the existing tests to always use both on-heap and off-heap vectors. Testing and diagnoses was done locally.

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

Closes #19367 from hvanhovell/SPARK-22143.
2017-09-27 23:08:30 +02:00
Takuya UESHIN 09cbf3df20 [SPARK-22125][PYSPARK][SQL] Enable Arrow Stream format for vectorized UDF.
## What changes were proposed in this pull request?

Currently we use Arrow File format to communicate with Python worker when invoking vectorized UDF but we can use Arrow Stream format.

This pr replaces the Arrow File format with the Arrow Stream format.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #19349 from ueshin/issues/SPARK-22125.
2017-09-27 23:21:44 +09:00
guoxiaolong d2b8b63b93 [SAPRK-20785][WEB-UI][SQL] Spark should provide jump links and add (count) in the SQL web ui.
## What changes were proposed in this pull request?

propose:

it provide links that jump to Running Queries,Completed Queries and Failed Queries.
it add (count) about Running Queries,Completed Queries and Failed Queries.
This is a small optimization in in the SQL web ui.

fix before:

![1](https://user-images.githubusercontent.com/26266482/30840686-36025cc0-a2ab-11e7-8d8d-1de0122a84fb.png)

fix after:
![2](https://user-images.githubusercontent.com/26266482/30840723-6cc67a52-a2ab-11e7-8002-9191a55895a6.png)

## How was this patch tested?

manual tests

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

Author: guoxiaolong <guo.xiaolong1@zte.com.cn>

Closes #19346 from guoxiaolongzte/SPARK-20785.
2017-09-27 20:48:55 +08:00
Wang Gengliang 9c5935d00b [SPARK-22141][SQL] Propagate empty relation before checking Cartesian products
## What changes were proposed in this pull request?

When inferring constraints from children, Join's condition can be simplified as None.
For example,
```
val testRelation = LocalRelation('a.int)
val x = testRelation.as("x")
val y = testRelation.where($"a" === 2 && !($"a" === 2)).as("y")
x.join.where($"x.a" === $"y.a")
```
The plan will become
```
Join Inner
:- LocalRelation <empty>, [a#23]
+- LocalRelation <empty>, [a#224]
```
And the Cartesian products check will throw exception for above plan.

Propagate empty relation before checking Cartesian products, and the issue is resolved.

## How was this patch tested?

Unit test

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #19362 from gengliangwang/MoveCheckCartesianProducts.
2017-09-27 12:44:10 +02:00
Juliusz Sompolski f21f6ce998 [SPARK-22103][FOLLOWUP] Rename addExtraCode to addInnerClass
## What changes were proposed in this pull request?

Address PR comments that appeared post-merge, to rename `addExtraCode` to `addInnerClass`,
and not count the size of the inner class to the size of the outer class.

## How was this patch tested?

YOLO.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #19353 from juliuszsompolski/SPARK-22103followup.
2017-09-26 10:04:34 -07:00
Liang-Chi Hsieh 64fbd1cef3 [SPARK-22124][SQL] Sample and Limit should also defer input evaluation under codegen
## What changes were proposed in this pull request?

We can override `usedInputs` to claim that an operator defers input evaluation. `Sample` and `Limit` are two operators which should claim it but don't. We should do it.

## How was this patch tested?

Existing tests.

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

Closes #19345 from viirya/SPARK-22124.
2017-09-26 15:23:13 +08:00
Bryan Cutler d8e825e3bc [SPARK-22106][PYSPARK][SQL] Disable 0-parameter pandas_udf and add doctests
## What changes were proposed in this pull request?

This change disables the use of 0-parameter pandas_udfs due to the API being overly complex and awkward, and can easily be worked around by using an index column as an input argument.  Also added doctests for pandas_udfs which revealed bugs for handling empty partitions and using the pandas_udf decorator.

## How was this patch tested?

Reworked existing 0-parameter test to verify error is raised, added doctest for pandas_udf, added new tests for empty partition and decorator usage.

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #19325 from BryanCutler/arrow-pandas_udf-0-param-remove-SPARK-22106.
2017-09-26 10:54:00 +09:00
Juliusz Sompolski 038b185736 [SPARK-22103] Move HashAggregateExec parent consume to a separate function in codegen
## What changes were proposed in this pull request?

HashAggregateExec codegen uses two paths for fast hash table and a generic one.
It generates code paths for iterating over both, and both code paths generate the consume code of the parent operator, resulting in that code being expanded twice.
This leads to a long generated function that might be an issue for the compiler (see e.g. SPARK-21603).
I propose to remove the double expansion by generating the consume code in a helper function that can just be called from both iterating loops.

An issue with separating the `consume` code to a helper function was that a number of places relied and assumed on being in the scope of an outside `produce` loop and e.g. use `continue` to jump out.
I replaced such code flows with nested scopes. It is code that should be handled the same by compiler, while getting rid of depending on assumptions that are outside of the `consume`'s own scope.

## How was this patch tested?

Existing test coverage.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #19324 from juliuszsompolski/aggrconsumecodegen.
2017-09-25 12:50:25 -07:00
Zhenhua Wang 365a29bdbf [SPARK-22100][SQL] Make percentile_approx support date/timestamp type and change the output type to be the same as input type
## What changes were proposed in this pull request?

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

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

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

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

## How was this patch tested?

Added a new test and modified some existing tests.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #19321 from wzhfy/approx_percentile_support_types.
2017-09-25 09:28:42 -07:00
Sean Owen 576c43fb42 [SPARK-22087][SPARK-14650][WIP][BUILD][REPL][CORE] Compile Spark REPL for Scala 2.12 + other 2.12 fixes
## What changes were proposed in this pull request?

Enable Scala 2.12 REPL. Fix most remaining issues with 2.12 compilation and warnings, including:

- Selecting Kafka 0.10.1+ for Scala 2.12 and patching over a minor API difference
- Fixing lots of "eta expansion of zero arg method deprecated" warnings
- Resolving the SparkContext.sequenceFile implicits compile problem
- Fixing an odd but valid jetty-server missing dependency in hive-thriftserver

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #19307 from srowen/Scala212.
2017-09-24 09:40:13 +01:00
Liang-Chi Hsieh 2274d84efc [SPARK-21338][SQL][FOLLOW-UP] Implement isCascadingTruncateTable() method in AggregatedDialect
## What changes were proposed in this pull request?

The implemented `isCascadingTruncateTable` in `AggregatedDialect` is wrong. When no dialect claims cascading, once there is an unknown cascading truncate in the dialects, we should return unknown cascading, instead of false.

## How was this patch tested?

Added test.

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

Closes #19286 from viirya/SPARK-21338-followup.
2017-09-23 21:51:04 -07:00
hyukjinkwon 04975a68b5 [SPARK-22109][SQL] Resolves type conflicts between strings and timestamps in partition column
## What changes were proposed in this pull request?

This PR proposes to resolve the type conflicts in strings and timestamps in partition column values.
It looks we need to set the timezone as it needs a cast between strings and timestamps.

```scala
val df = Seq((1, "2015-01-01 00:00:00"), (2, "2014-01-01 00:00:00"), (3, "blah")).toDF("i", "str")
val path = "/tmp/test.parquet"
df.write.format("parquet").partitionBy("str").save(path)
spark.read.parquet(path).show()
```

**Before**

```
java.util.NoSuchElementException: None.get
  at scala.None$.get(Option.scala:347)
  at scala.None$.get(Option.scala:345)
  at org.apache.spark.sql.catalyst.expressions.TimeZoneAwareExpression$class.timeZone(datetimeExpressions.scala:46)
  at org.apache.spark.sql.catalyst.expressions.Cast.timeZone$lzycompute(Cast.scala:172)
  at org.apache.spark.sql.catalyst.expressions.Cast.timeZone(Cast.scala:172)
  at org.apache.spark.sql.catalyst.expressions.Cast$$anonfun$castToString$3$$anonfun$apply$16.apply(Cast.scala:208)
  at org.apache.spark.sql.catalyst.expressions.Cast$$anonfun$castToString$3$$anonfun$apply$16.apply(Cast.scala:208)
  at org.apache.spark.sql.catalyst.expressions.Cast.org$apache$spark$sql$catalyst$expressions$Cast$$buildCast(Cast.scala:201)
  at org.apache.spark.sql.catalyst.expressions.Cast$$anonfun$castToString$3.apply(Cast.scala:207)
  at org.apache.spark.sql.catalyst.expressions.Cast.nullSafeEval(Cast.scala:533)
  at org.apache.spark.sql.catalyst.expressions.UnaryExpression.eval(Expression.scala:331)
  at org.apache.spark.sql.execution.datasources.PartitioningUtils$$anonfun$org$apache$spark$sql$execution$datasources$PartitioningUtils$$resolveTypeConflicts$1.apply(PartitioningUtils.scala:481)
  at org.apache.spark.sql.execution.datasources.PartitioningUtils$$anonfun$org$apache$spark$sql$execution$datasources$PartitioningUtils$$resolveTypeConflicts$1.apply(PartitioningUtils.scala:480)
  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)
```

**After**

```
+---+-------------------+
|  i|                str|
+---+-------------------+
|  2|2014-01-01 00:00:00|
|  1|2015-01-01 00:00:00|
|  3|               blah|
+---+-------------------+
```

## How was this patch tested?

Unit tests added in `ParquetPartitionDiscoverySuite` and manual tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #19331 from HyukjinKwon/SPARK-22109.
2017-09-24 00:05:17 +09:00
Sean Owen 50ada2a4d3 [SPARK-22033][CORE] BufferHolder, other size checks should account for the specific VM array size limitations
## What changes were proposed in this pull request?

Try to avoid allocating an array bigger than Integer.MAX_VALUE - 8, which is the actual max size on some JVMs, in several places

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #19266 from srowen/SPARK-22033.
2017-09-23 15:40:59 +01:00
guoxiaolong 3920af7d1d [SPARK-22099] The 'job ids' list style needs to be changed in the SQL page.
## What changes were proposed in this pull request?

The 'job ids' list style needs to be changed in the SQL page. There are two reasons:
1. If a job id is a line, there are a lot of job ids, then the table row height will be high. As shown below:
![3](https://user-images.githubusercontent.com/26266482/30732242-2fb11442-9fa4-11e7-98ea-80a98f280243.png)

2. should be consistent with the 'JDBC / ODBC Server' page style, I am in this way to modify the style. As shown below:
![2](https://user-images.githubusercontent.com/26266482/30732257-3c550820-9fa4-11e7-9d8e-467d3011e0ac.png)

My changes are as follows:
![6](https://user-images.githubusercontent.com/26266482/30732318-8f61d8b8-9fa4-11e7-8af5-037ed12b13c9.png)

![5](https://user-images.githubusercontent.com/26266482/30732284-5b6a6c00-9fa4-11e7-8db9-3a2291f37ae6.png)

## How was this patch tested?
manual tests

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

Author: guoxiaolong <guo.xiaolong1@zte.com.cn>

Closes #19320 from guoxiaolongzte/SPARK-22099.
2017-09-23 15:39:53 +01:00
Ala Luszczak d2b2932d8b [SPARK-22092] Reallocation in OffHeapColumnVector.reserveInternal corrupts struct and array data
## What changes were proposed in this pull request?

`OffHeapColumnVector.reserveInternal()` will only copy already inserted values during reallocation if `data != null`. In vectors containing arrays or structs this is incorrect, since there field `data` is not used at all. We need to check `nulls` instead.

## How was this patch tested?

Adds new tests to `ColumnVectorSuite` that reproduce the errors.

Author: Ala Luszczak <ala@databricks.com>

Closes #19308 from ala/vector-realloc.
2017-09-22 15:31:43 +02:00
Bryan Cutler 27fc536d9a [SPARK-21190][PYSPARK] Python Vectorized UDFs
This PR adds vectorized UDFs to the Python API

**Proposed API**
Introduce a flag to turn on vectorization for a defined UDF, for example:

```
pandas_udf(DoubleType())
def plus(a, b)
    return a + b
```
or

```
plus = pandas_udf(lambda a, b: a + b, DoubleType())
```
Usage is the same as normal UDFs

0-parameter UDFs
pandas_udf functions can declare an optional `**kwargs` and when evaluated, will contain a key "size" that will give the required length of the output.  For example:

```
pandas_udf(LongType())
def f0(**kwargs):
    return pd.Series(1).repeat(kwargs["size"])

df.select(f0())
```

Added new unit tests in pyspark.sql that are enabled if pyarrow and Pandas are available.

- [x] Fix support for promoted types with null values
- [ ] Discuss 0-param UDF API (use of kwargs)
- [x] Add tests for chained UDFs
- [ ] Discuss behavior when pyarrow not installed / enabled
- [ ] Cleanup pydoc and add user docs

Author: Bryan Cutler <cutlerb@gmail.com>
Author: Takuya UESHIN <ueshin@databricks.com>

Closes #18659 from BryanCutler/arrow-vectorized-udfs-SPARK-21404.
2017-09-22 16:17:50 +08:00
maryannxue 5960686e79 [SPARK-21998][SQL] SortMergeJoinExec did not calculate its outputOrdering correctly during physical planning
## What changes were proposed in this pull request?

Right now the calculation of SortMergeJoinExec's outputOrdering relies on the fact that its children have already been sorted on the join keys, while this is often not true until EnsureRequirements has been applied. So we ended up not getting the correct outputOrdering during physical planning stage before Sort nodes are added to the children.

For example, J = {A join B on key1 = key2}
1. if A is NOT ordered on key1 ASC, J's outputOrdering should include "key1 ASC"
2. if A is ordered on key1 ASC, J's outputOrdering should include "key1 ASC"
3. if A is ordered on key1 ASC, with sameOrderExp=c1, J's outputOrdering should include "key1 ASC, sameOrderExp=c1"

So to fix this I changed the  behavior of <code>getKeyOrdering(keys, childOutputOrdering)</code> to:
1. If the childOutputOrdering satisfies (is a superset of) the required child ordering => childOutputOrdering
2. Otherwise => required child ordering

In addition, I organized the logic for deciding the relationship between two orderings into SparkPlan, so that it can be reused by EnsureRequirements and SortMergeJoinExec, and potentially other classes.

## How was this patch tested?

Added new test cases.
Passed all integration tests.

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

Closes #19281 from maryannxue/spark-21998.
2017-09-21 23:54:16 -07:00
Shixiong Zhu fedf6961be [SPARK-22094][SS] processAllAvailable should check the query state
## What changes were proposed in this pull request?

`processAllAvailable` should also check the query state and if the query is stopped, it should return.

## How was this patch tested?

The new unit test.

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #19314 from zsxwing/SPARK-22094.
2017-09-21 21:55:07 -07:00
Tathagata Das f32a842505 [SPARK-22053][SS] Stream-stream inner join in Append Mode
## What changes were proposed in this pull request?

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

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

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

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

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

Refer to the scaladocs class for more implementation details.

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

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

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

Closes #19271 from tdas/SPARK-22053.
2017-09-21 15:39:07 -07:00
Zheng RuiFeng a8a5cd24e2 [SPARK-22009][ML] Using treeAggregate improve some algs
## What changes were proposed in this pull request?

I test on a dataset of about 13M instances, and found that using `treeAggregate` give a speedup in following algs:

|Algs| SpeedUp |
|------|-----------|
|OneHotEncoder| 5% |
|StatFunctions.calculateCov| 7% |
|StatFunctions.multipleApproxQuantiles|  9% |
|RegressionEvaluator| 8% |

## How was this patch tested?
existing tests

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #19232 from zhengruifeng/use_treeAggregate.
2017-09-21 20:06:42 +01:00
Sean Owen f10cbf17dc [SPARK-21977][HOTFIX] Adjust EnsureStatefulOpPartitioningSuite to use scalatest lifecycle normally instead of constructor
## What changes were proposed in this pull request?

Adjust EnsureStatefulOpPartitioningSuite to use scalatest lifecycle normally instead of constructor; fixes:

```
*** RUN ABORTED ***
  org.apache.spark.SparkException: Only one SparkContext may be running in this JVM (see SPARK-2243). To ignore this error, set spark.driver.allowMultipleContexts = true. The currently running SparkContext was created at:
org.apache.spark.sql.streaming.EnsureStatefulOpPartitioningSuite.<init>(EnsureStatefulOpPartitioningSuite.scala:35)
```

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #19306 from srowen/SPARK-21977.2.
2017-09-21 18:00:19 +01:00
Sean Owen e17901d6df [SPARK-22049][DOCS] Confusing behavior of from_utc_timestamp and to_utc_timestamp
## What changes were proposed in this pull request?

Clarify behavior of to_utc_timestamp/from_utc_timestamp with an example

## How was this patch tested?

Doc only change / existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #19276 from srowen/SPARK-22049.
2017-09-20 20:47:17 +09:00
Sean Owen 3d4dd14cd5 [SPARK-22066][BUILD] Update checkstyle to 8.2, enable it, fix violations
## What changes were proposed in this pull request?

Update plugins, including scala-maven-plugin, to latest versions. Update checkstyle to 8.2. Remove bogus checkstyle config and enable it. Fix existing and new Java checkstyle errors.

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #19282 from srowen/SPARK-22066.
2017-09-20 10:01:46 +01:00
Burak Yavuz 280ff523f4 [SPARK-21977] SinglePartition optimizations break certain Streaming Stateful Aggregation requirements
## What changes were proposed in this pull request?

This is a bit hard to explain as there are several issues here, I'll try my best. Here are the requirements:
  1. A StructuredStreaming Source that can generate empty RDDs with 0 partitions
  2. A StructuredStreaming query that uses the above source, performs a stateful aggregation
     (mapGroupsWithState, groupBy.count, ...), and coalesce's by 1

The crux of the problem is that when a dataset has a `coalesce(1)` call, it receives a `SinglePartition` partitioning scheme. This scheme satisfies most required distributions used for aggregations such as HashAggregateExec. This causes a world of problems:
  Symptom 1. If the input RDD has 0 partitions, the whole lineage will receive 0 partitions, nothing will be executed, the state store will not create any delta files. When this happens, the next trigger fails, because the StateStore fails to load the delta file for the previous trigger
  Symptom 2. Let's say that there was data. Then in this case, if you stop your stream, and change `coalesce(1)` with `coalesce(2)`, then restart your stream, your stream will fail, because `spark.sql.shuffle.partitions - 1` number of StateStores will fail to find its delta files.

To fix the issues above, we must check that the partitioning of the child of a `StatefulOperator` satisfies:
If the grouping expressions are empty:
  a) AllTuple distribution
  b) Single physical partition
If the grouping expressions are non empty:
  a) Clustered distribution
  b) spark.sql.shuffle.partition # of partitions
whether or not `coalesce(1)` exists in the plan, and whether or not the input RDD for the trigger has any data.

Once you fix the above problem by adding an Exchange to the plan, you come across the following bug:
If you call `coalesce(1).groupBy().count()` on a Streaming DataFrame, and if you have a trigger with no data, `StateStoreRestoreExec` doesn't return the prior state. However, for this specific aggregation, `HashAggregateExec` after the restore returns a (0, 0) row, since we're performing a count, and there is no data. Then this data gets stored in `StateStoreSaveExec` causing the previous counts to be overwritten and lost.

## How was this patch tested?

Regression tests

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #19196 from brkyvz/sa-0.
2017-09-20 00:01:21 -07:00
Marcelo Vanzin c6ff59a230 [SPARK-18838][CORE] Add separate listener queues to LiveListenerBus.
This change modifies the live listener bus so that all listeners are
added to queues; each queue has its own thread to dispatch events,
making it possible to separate slow listeners from other more
performance-sensitive ones.

The public API has not changed - all listeners added with the existing
"addListener" method, which after this change mostly means all
user-defined listeners, end up in a default queue. Internally, there's
an API allowing listeners to be added to specific queues, and that API
is used to separate the internal Spark listeners into 3 categories:
application status listeners (e.g. UI), executor management (e.g. dynamic
allocation), and the event log.

The queueing logic, while abstracted away in a separate class, is kept
as much as possible hidden away from consumers. Aside from choosing their
queue, there's no code change needed to take advantage of queues.

Test coverage relies on existing tests; a few tests had to be tweaked
because they relied on `LiveListenerBus.postToAll` being synchronous,
and the change makes that method asynchronous. Other tests were simplified
not to use the asynchronous LiveListenerBus.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19211 from vanzin/SPARK-18838.
2017-09-20 13:41:29 +08:00
Bryan Cutler 718bbc9390 [SPARK-22067][SQL] ArrowWriter should use position when setting UTF8String ByteBuffer
## What changes were proposed in this pull request?

The ArrowWriter StringWriter was setting Arrow data using a position of 0 instead of the actual position in the ByteBuffer.  This was currently working because of a bug ARROW-1443, and has been fixed as of
Arrow 0.7.0.  Testing with this version revealed the error in ArrowConvertersSuite test string conversion.

## How was this patch tested?

Existing tests, manually verified working with Arrow 0.7.0

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #19284 from BryanCutler/arrow-ArrowWriter-StringWriter-position-SPARK-22067.
2017-09-20 10:51:00 +09:00
aokolnychyi ee13f3e3dc [SPARK-21969][SQL] CommandUtils.updateTableStats should call refreshTable
## What changes were proposed in this pull request?

Tables in the catalog cache are not invalidated once their statistics are updated. As a consequence, existing sessions will use the cached information even though it is not valid anymore. Consider and an example below.

```
// step 1
spark.range(100).write.saveAsTable("tab1")
// step 2
spark.sql("analyze table tab1 compute statistics")
// step 3
spark.sql("explain cost select distinct * from tab1").show(false)
// step 4
spark.range(100).write.mode("append").saveAsTable("tab1")
// step 5
spark.sql("explain cost select distinct * from tab1").show(false)
```

After step 3, the table will be present in the catalog relation cache. Step 4 will correctly update the metadata inside the catalog but will NOT invalidate the cache.

By the way, ``spark.sql("analyze table tab1 compute statistics")`` between step 3 and step 4 would also solve the problem.

## How was this patch tested?

Current and additional unit tests.

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

Closes #19252 from aokolnychyi/spark-21969.
2017-09-19 14:19:13 -07:00
Huaxin Gao d5aefa83ad [SPARK-21338][SQL] implement isCascadingTruncateTable() method in AggregatedDialect
## What changes were proposed in this pull request?

org.apache.spark.sql.jdbc.JdbcDialect's method:
def isCascadingTruncateTable(): Option[Boolean] = None
is not overriden in org.apache.spark.sql.jdbc.AggregatedDialect class.
Because of this issue, when you add more than one dialect Spark doesn't truncate table because isCascadingTruncateTable always returns default None for Aggregated Dialect.
Will implement isCascadingTruncateTable in AggregatedDialect class in this PR.

## How was this patch tested?

In JDBCSuite, inside test("Aggregated dialects"), will add one line to test AggregatedDialect.isCascadingTruncateTable

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

Closes #19256 from huaxingao/spark-21338.
2017-09-19 09:27:05 -07:00
Taaffy 1bc17a6b8a [SPARK-22052] Incorrect Metric assigned in MetricsReporter.scala
Current implementation for processingRate-total uses wrong metric:
mistakenly uses inputRowsPerSecond instead of processedRowsPerSecond

## What changes were proposed in this pull request?
Adjust processingRate-total from using inputRowsPerSecond to processedRowsPerSecond

## How was this patch tested?

Built spark from source with proposed change and tested output with correct parameter. Before change the csv metrics file for inputRate-total and processingRate-total displayed the same values due to the error. After changing MetricsReporter.scala the processingRate-total csv file displayed the correct metric.
<img width="963" alt="processed rows per second" src="https://user-images.githubusercontent.com/32072374/30554340-82eea12c-9ca4-11e7-8370-8168526ff9a2.png">

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

Author: Taaffy <32072374+Taaffy@users.noreply.github.com>

Closes #19268 from Taaffy/patch-1.
2017-09-19 10:20:04 +01:00
Kevin Yu c66d64b3df [SPARK-14878][SQL] Trim characters string function support
#### What changes were proposed in this pull request?

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

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

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

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

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

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

Closes #12646 from kevinyu98/spark-14878.
2017-09-18 12:12:35 -07:00
Feng Liu 3b049abf10 [SPARK-22003][SQL] support array column in vectorized reader with UDF
## What changes were proposed in this pull request?

The UDF needs to deserialize the `UnsafeRow`. When the column type is Array, the `get` method from the `ColumnVector`, which is used by the vectorized reader, is called, but this method is not implemented.

## 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 #19230 from liufengdb/fix_array_open.
2017-09-18 08:49:32 -07:00
Jose Torres 0bad10d3e3 [SPARK-22017] Take minimum of all watermark execs in StreamExecution.
## What changes were proposed in this pull request?

Take the minimum of all watermark exec nodes as the "real" watermark in StreamExecution, rather than picking one arbitrarily.

## How was this patch tested?

new unit test

Author: Jose Torres <jose@databricks.com>

Closes #19239 from joseph-torres/SPARK-22017.
2017-09-15 21:10:07 -07:00
Wenchen Fan c7307acdad [SPARK-15689][SQL] data source v2 read path
## What changes were proposed in this pull request?

This PR adds the infrastructure for data source v2, and implement features which Spark already have in data source v1, i.e. column pruning, filter push down, catalyst expression filter push down, InternalRow scan, schema inference, data size report. The write path is excluded to avoid making this PR growing too big, and will be added in follow-up PR.

## How was this patch tested?

new tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19136 from cloud-fan/data-source-v2.
2017-09-15 22:18:36 +08:00
Wenchen Fan 3c6198c86e [SPARK-21987][SQL] fix a compatibility issue of sql event logs
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/18600 we removed the `metadata` field from `SparkPlanInfo`. This causes a problem when we replay event logs that are generated by older Spark versions.

## How was this patch tested?

a regression test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19237 from cloud-fan/event.
2017-09-15 00:47:44 -07:00
Yuming Wang 4decedfdbd [SPARK-22002][SQL] Read JDBC table use custom schema support specify partial fields.
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/18266 add a new feature to support read JDBC table use custom schema, but we must specify all the fields. For simplicity, this PR support  specify partial fields.

## How was this patch tested?
unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #19231 from wangyum/SPARK-22002.
2017-09-14 23:35:55 -07:00
goldmedal a28728a9af [SPARK-21513][SQL][FOLLOWUP] Allow UDF to_json support converting MapType to json for PySpark and SparkR
## What changes were proposed in this pull request?
In previous work SPARK-21513, we has allowed `MapType` and `ArrayType` of `MapType`s convert to a json string but only for Scala API. In this follow-up PR, we will make SparkSQL support it for PySpark and SparkR, too. We also fix some little bugs and comments of the previous work in this follow-up PR.

### For PySpark
```
>>> data = [(1, {"name": "Alice"})]
>>> df = spark.createDataFrame(data, ("key", "value"))
>>> df.select(to_json(df.value).alias("json")).collect()
[Row(json=u'{"name":"Alice")']
>>> data = [(1, [{"name": "Alice"}, {"name": "Bob"}])]
>>> df = spark.createDataFrame(data, ("key", "value"))
>>> df.select(to_json(df.value).alias("json")).collect()
[Row(json=u'[{"name":"Alice"},{"name":"Bob"}]')]
```
### For SparkR
```
# Converts a map into a JSON object
df2 <- sql("SELECT map('name', 'Bob')) as people")
df2 <- mutate(df2, people_json = to_json(df2$people))
# Converts an array of maps into a JSON array
df2 <- sql("SELECT array(map('name', 'Bob'), map('name', 'Alice')) as people")
df2 <- mutate(df2, people_json = to_json(df2$people))
```
## How was this patch tested?
Add unit test cases.

cc viirya HyukjinKwon

Author: goldmedal <liugs963@gmail.com>

Closes #19223 from goldmedal/SPARK-21513-fp-PySaprkAndSparkR.
2017-09-15 11:53:10 +09:00
Jose Torres 054ddb2f54 [SPARK-21988] Add default stats to StreamingExecutionRelation.
## What changes were proposed in this pull request?

Add default stats to StreamingExecutionRelation.

## How was this patch tested?

existing unit tests and an explain() test to be sure

Author: Jose Torres <jose@databricks.com>

Closes #19212 from joseph-torres/SPARK-21988.
2017-09-14 11:06:25 -07:00
Zhenhua Wang ddd7f5e11d [SPARK-17642][SQL][FOLLOWUP] drop test tables and improve comments
## What changes were proposed in this pull request?

Drop test tables and improve comments.

## How was this patch tested?

Modified existing test.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #19213 from wzhfy/useless_comment.
2017-09-14 23:14:21 +08:00
gatorsmile 4e6fc69014 [SPARK-4131][FOLLOW-UP] Support "Writing data into the filesystem from queries"
## What changes were proposed in this pull request?
This PR is clean the codes in https://github.com/apache/spark/pull/18975

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19225 from gatorsmile/refactorSPARK-4131.
2017-09-14 14:48:04 +08:00
Takeshi Yamamuro 8be7e6bb3c [SPARK-21973][SQL] Add an new option to filter queries in TPC-DS
## What changes were proposed in this pull request?
This pr added a new option to filter TPC-DS queries to run in `TPCDSQueryBenchmark`.
By default, `TPCDSQueryBenchmark` runs all the TPC-DS queries.
This change could enable developers to run some of the TPC-DS queries by this option,
e.g., to run q2, q4, and q6 only:
```
spark-submit --class <this class> --conf spark.sql.tpcds.queryFilter="q2,q4,q6" --jars <spark sql test jar>
```

## How was this patch tested?
Manually checked.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #19188 from maropu/RunPartialQueriesInTPCDS.
2017-09-13 21:54:10 -07:00
Yuming Wang 17edfec59d [SPARK-20427][SQL] Read JDBC table use custom schema
## What changes were proposed in this pull request?

Auto generated Oracle schema some times not we expect:

- `number(1)` auto mapped to BooleanType, some times it's not we expect, per [SPARK-20921](https://issues.apache.org/jira/browse/SPARK-20921).
-  `number` auto mapped to Decimal(38,10), It can't read big data, per [SPARK-20427](https://issues.apache.org/jira/browse/SPARK-20427).

This PR fix this issue by custom schema as follows:
```scala
val props = new Properties()
props.put("customSchema", "ID decimal(38, 0), N1 int, N2 boolean")
val dfRead = spark.read.schema(schema).jdbc(jdbcUrl, "tableWithCustomSchema", props)
dfRead.show()
```
or
```sql
CREATE TEMPORARY VIEW tableWithCustomSchema
USING org.apache.spark.sql.jdbc
OPTIONS (url '$jdbcUrl', dbTable 'tableWithCustomSchema', customSchema'ID decimal(38, 0), N1 int, N2 boolean')
```

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18266 from wangyum/SPARK-20427.
2017-09-13 16:34:17 -07:00
donnyzone 21c4450fb2 [SPARK-21980][SQL] References in grouping functions should be indexed with semanticEquals
## What changes were proposed in this pull request?

https://issues.apache.org/jira/browse/SPARK-21980

This PR fixes the issue in ResolveGroupingAnalytics rule, which indexes the column references in grouping functions without considering case sensitive configurations.

The problem can be reproduced by:

`val df = spark.createDataFrame(Seq((1, 1), (2, 1), (2, 2))).toDF("a", "b")
 df.cube("a").agg(grouping("A")).show()`

## How was this patch tested?
unit tests

Author: donnyzone <wellfengzhu@gmail.com>

Closes #19202 from DonnyZone/ResolveGroupingAnalytics.
2017-09-13 10:06:53 -07:00
Armin b6ef1f57bc [SPARK-21970][CORE] Fix Redundant Throws Declarations in Java Codebase
## What changes were proposed in this pull request?

1. Removing all redundant throws declarations from Java codebase.
2. Removing dead code made visible by this from `ShuffleExternalSorter#closeAndGetSpills`

## How was this patch tested?

Build still passes.

Author: Armin <me@obrown.io>

Closes #19182 from original-brownbear/SPARK-21970.
2017-09-13 14:04:26 +01:00
goldmedal 371e4e2053 [SPARK-21513][SQL] Allow UDF to_json support converting MapType to json
# What changes were proposed in this pull request?
UDF to_json only supports converting `StructType` or `ArrayType` of `StructType`s to a json output string now.
According to the discussion of JIRA SPARK-21513, I allow to `to_json` support converting `MapType` and `ArrayType` of `MapType`s to a json output string.
This PR is for SQL and Scala API only.

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

cc viirya HyukjinKwon

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

Closes #18875 from goldmedal/SPARK-21513.
2017-09-13 09:43:00 +09:00
sarutak b9b54b1c88 [SPARK-21368][SQL] TPCDSQueryBenchmark can't refer query files.
## What changes were proposed in this pull request?

TPCDSQueryBenchmark packaged into a jar doesn't work with spark-submit.
It's because of the failure of reference query files in the jar file.

## How was this patch tested?

Ran the benchmark.

Author: sarutak <sarutak@oss.nttdata.co.jp>
Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #18592 from sarutak/fix-tpcds-benchmark.
2017-09-12 10:49:46 -07:00
Zhenhua Wang 515910e9bd [SPARK-17642][SQL] support DESC EXTENDED/FORMATTED table column commands
## What changes were proposed in this pull request?

Support DESC (EXTENDED | FORMATTED) ? TABLE COLUMN command.
Support DESC EXTENDED | FORMATTED TABLE COLUMN command to show column-level statistics.
Do NOT support describe nested columns.

## How was this patch tested?

Added test cases.

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

Closes #16422 from wzhfy/descColumn.
2017-09-12 08:59:52 -07:00
Jen-Ming Chung 7d0a3ef4ce [SPARK-21610][SQL][FOLLOWUP] Corrupt records are not handled properly when creating a dataframe from a file
## What changes were proposed in this pull request?

When the `requiredSchema` only contains `_corrupt_record`, the derived `actualSchema` is empty and the `_corrupt_record` are all null for all rows. This PR captures above situation and raise an exception with a reasonable workaround messag so that users can know what happened and how to fix the query.

## How was this patch tested?

Added unit test in `CSVSuite`.

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

Closes #19199 from jmchung/SPARK-21610-FOLLOWUP.
2017-09-12 22:47:12 +09:00
caoxuewen dc74c0e67d [MINOR][SQL] remove unuse import class
## What changes were proposed in this pull request?

this PR describe remove the import class that are unused.

## How was this patch tested?

N/A

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

Closes #19131 from heary-cao/unuse_import.
2017-09-11 10:09:20 +01:00
Jen-Ming Chung 6273a711b6 [SPARK-21610][SQL] Corrupt records are not handled properly when creating a dataframe from a file
## What changes were proposed in this pull request?
```
echo '{"field": 1}
{"field": 2}
{"field": "3"}' >/tmp/sample.json
```

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

val schema = new StructType()
  .add("field", ByteType)
  .add("_corrupt_record", StringType)

val file = "/tmp/sample.json"

val dfFromFile = spark.read.schema(schema).json(file)

scala> dfFromFile.show(false)
+-----+---------------+
|field|_corrupt_record|
+-----+---------------+
|1    |null           |
|2    |null           |
|null |{"field": "3"} |
+-----+---------------+

scala> dfFromFile.filter($"_corrupt_record".isNotNull).count()
res1: Long = 0

scala> dfFromFile.filter($"_corrupt_record".isNull).count()
res2: Long = 3
```
When the `requiredSchema` only contains `_corrupt_record`, the derived `actualSchema` is empty and the `_corrupt_record` are all null for all rows. This PR captures above situation and raise an exception with a reasonable workaround messag so that users can know what happened and how to fix the query.

## How was this patch tested?

Added test case.

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

Closes #18865 from jmchung/SPARK-21610.
2017-09-10 17:26:43 -07:00
Jane Wang f76790557b [SPARK-4131] Support "Writing data into the filesystem from queries"
## What changes were proposed in this pull request?

This PR implements the sql feature:
INSERT OVERWRITE [LOCAL] DIRECTORY directory1
  [ROW FORMAT row_format] [STORED AS file_format]
  SELECT ... FROM ...

## How was this patch tested?
Added new unittests and also pulled the code to fb-spark so that we could test writing to hdfs directory.

Author: Jane Wang <janewang@fb.com>

Closes #18975 from janewangfb/port_local_directory.
2017-09-09 11:48:34 -07:00
Yanbo Liang e4d8f9a36a [MINOR][SQL] Correct DataFrame doc.
## What changes were proposed in this pull request?
Correct DataFrame doc.

## How was this patch tested?
Only doc change, no tests.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #19173 from yanboliang/df-doc.
2017-09-09 09:25:12 -07:00
Liang-Chi Hsieh 6b45d7e941 [SPARK-21954][SQL] JacksonUtils should verify MapType's value type instead of key type
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

Added tests.

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

Closes #19167 from viirya/test-jacksonutils.
2017-09-09 19:10:52 +09:00
Andrew Ash 8a5eb50681 [SPARK-21941] Stop storing unused attemptId in SQLTaskMetrics
## What changes were proposed in this pull request?

In a driver heap dump containing 390,105 instances of SQLTaskMetrics this
would have saved me approximately 3.2MB of memory.

Since we're not getting any benefit from storing this unused value, let's
eliminate it until a future PR makes use of it.

## How was this patch tested?

Existing unit tests

Author: Andrew Ash <andrew@andrewash.com>

Closes #19153 from ash211/aash/trim-sql-listener.
2017-09-08 23:33:15 -07:00
Kazuaki Ishizaki 8a4f228dc0 [SPARK-21946][TEST] fix flaky test: "alter table: rename cached table" in InMemoryCatalogedDDLSuite
## What changes were proposed in this pull request?

This PR fixes flaky test `InMemoryCatalogedDDLSuite "alter table: rename cached table"`.
Since this test validates distributed DataFrame, the result should be checked by using `checkAnswer`. The original version used `df.collect().Seq` method that does not guaranty an order of each element of the result.

## How was this patch tested?

Use existing test case

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

Closes #19159 from kiszk/SPARK-21946.
2017-09-08 09:39:20 -07:00
Dongjoon Hyun c26976fe14 [SPARK-21939][TEST] Use TimeLimits instead of Timeouts
Since ScalaTest 3.0.0, `org.scalatest.concurrent.Timeouts` is deprecated.
This PR replaces the deprecated one with `org.scalatest.concurrent.TimeLimits`.

```scala
-import org.scalatest.concurrent.Timeouts._
+import org.scalatest.concurrent.TimeLimits._
```

Pass the existing test suites.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19150 from dongjoon-hyun/SPARK-21939.

Change-Id: I1a1b07f1b97e51e2263dfb34b7eaaa099b2ded5e
2017-09-08 09:31:13 +08:00
Dongjoon Hyun eea2b877cf [SPARK-21912][SQL] ORC/Parquet table should not create invalid column names
## What changes were proposed in this pull request?

Currently, users meet job abortions while creating or altering ORC/Parquet tables with invalid column names. We had better prevent this by raising **AnalysisException** with a guide to use aliases instead like Paquet data source tables.

**BEFORE**
```scala
scala> sql("CREATE TABLE orc1 USING ORC AS SELECT 1 `a b`")
17/09/04 13:28:21 ERROR Utils: Aborting task
java.lang.IllegalArgumentException: Error: : expected at the position 8 of 'struct<a b:int>' but ' ' is found.
17/09/04 13:28:21 ERROR FileFormatWriter: Job job_20170904132821_0001 aborted.
17/09/04 13:28:21 ERROR Executor: Exception in task 0.0 in stage 1.0 (TID 1)
org.apache.spark.SparkException: Task failed while writing rows.
```

**AFTER**
```scala
scala> sql("CREATE TABLE orc1 USING ORC AS SELECT 1 `a b`")
17/09/04 13:27:40 ERROR CreateDataSourceTableAsSelectCommand: Failed to write to table orc1
org.apache.spark.sql.AnalysisException: Attribute name "a b" contains invalid character(s) among " ,;{}()\n\t=". Please use alias to rename it.;
```

## How was this patch tested?

Pass the Jenkins with a new test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19124 from dongjoon-hyun/SPARK-21912.
2017-09-06 22:20:48 -07:00
Liang-Chi Hsieh ce7293c150 [SPARK-21835][SQL][FOLLOW-UP] RewritePredicateSubquery should not produce unresolved query plans
## What changes were proposed in this pull request?

This is a follow-up of #19050 to deal with `ExistenceJoin` case.

## How was this patch tested?

Added test.

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

Closes #19151 from viirya/SPARK-21835-followup.
2017-09-06 22:15:25 -07:00
Jacek Laskowski fa0092bddf [SPARK-21901][SS] Define toString for StateOperatorProgress
## What changes were proposed in this pull request?

Just `StateOperatorProgress.toString` + few formatting fixes

## How was this patch tested?

Local build. Waiting for OK from Jenkins.

Author: Jacek Laskowski <jacek@japila.pl>

Closes #19112 from jaceklaskowski/SPARK-21901-StateOperatorProgress-toString.
2017-09-06 15:48:48 -07:00
Jose Torres acdf45fb52 [SPARK-21765] Check that optimization doesn't affect isStreaming bit.
## What changes were proposed in this pull request?

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

## How was this patch tested?

new and existing unit tests

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

Closes #19056 from joseph-torres/SPARK-21765-followup.
2017-09-06 11:19:46 -07:00
Liang-Chi Hsieh f2e22aebfe [SPARK-21835][SQL] RewritePredicateSubquery should not produce unresolved query plans
## What changes were proposed in this pull request?

Correlated predicate subqueries are rewritten into `Join` by the rule `RewritePredicateSubquery`  during optimization.

It is possibly that the two sides of the `Join` have conflicting attributes. The query plans produced by `RewritePredicateSubquery` become unresolved and break structural integrity.

We should check if there are conflicting attributes in the `Join` and de-duplicate them by adding a `Project`.

## How was this patch tested?

Added tests.

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

Closes #19050 from viirya/SPARK-21835.
2017-09-06 07:42:19 -07:00
Xingbo Jiang fd60d4fa6c [SPARK-21652][SQL] Fix rule confliction between InferFiltersFromConstraints and ConstantPropagation
## What changes were proposed in this pull request?

For the given example below, the predicate added by `InferFiltersFromConstraints` is folded by `ConstantPropagation` later, this leads to unconverged optimize iteration:
```
Seq((1, 1)).toDF("col1", "col2").createOrReplaceTempView("t1")
Seq(1, 2).toDF("col").createOrReplaceTempView("t2")
sql("SELECT * FROM t1, t2 WHERE t1.col1 = 1 AND 1 = t1.col2 AND t1.col1 = t2.col AND t1.col2 = t2.col")
```

We can fix this by adjusting the indent of the optimize rules.

## How was this patch tested?

Add test case that would have failed in `SQLQuerySuite`.

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

Closes #19099 from jiangxb1987/unconverge-optimization.
2017-09-05 13:12:39 -07:00
gatorsmile 2974406d17 [SPARK-21845][SQL][TEST-MAVEN] Make codegen fallback of expressions configurable
## What changes were proposed in this pull request?
We should make codegen fallback of expressions configurable. So far, it is always on. We might hide it when our codegen have compilation bugs. Thus, we should also disable the codegen fallback when running test cases.

## How was this patch tested?
Added test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19119 from gatorsmile/fallbackCodegen.
2017-09-05 09:04:03 -07:00
hyukjinkwon 02a4386aec [SPARK-20978][SQL] Bump up Univocity version to 2.5.4
## What changes were proposed in this pull request?

There was a bug in Univocity Parser that causes the issue in SPARK-20978. This was fixed as below:

```scala
val df = spark.read.schema("a string, b string, unparsed string").option("columnNameOfCorruptRecord", "unparsed").csv(Seq("a").toDS())
df.show()
```

**Before**

```
java.lang.NullPointerException
	at scala.collection.immutable.StringLike$class.stripLineEnd(StringLike.scala:89)
	at scala.collection.immutable.StringOps.stripLineEnd(StringOps.scala:29)
	at org.apache.spark.sql.execution.datasources.csv.UnivocityParser.org$apache$spark$sql$execution$datasources$csv$UnivocityParser$$getCurrentInput(UnivocityParser.scala:56)
	at org.apache.spark.sql.execution.datasources.csv.UnivocityParser$$anonfun$org$apache$spark$sql$execution$datasources$csv$UnivocityParser$$convert$1.apply(UnivocityParser.scala:207)
	at org.apache.spark.sql.execution.datasources.csv.UnivocityParser$$anonfun$org$apache$spark$sql$execution$datasources$csv$UnivocityParser$$convert$1.apply(UnivocityParser.scala:207)
...
```

**After**

```
+---+----+--------+
|  a|   b|unparsed|
+---+----+--------+
|  a|null|       a|
+---+----+--------+
```

It was fixed in 2.5.0 and 2.5.4 was released. I guess it'd be safe to upgrade this.

## How was this patch tested?

Unit test added in `CSVSuite.scala`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #19113 from HyukjinKwon/bump-up-univocity.
2017-09-05 23:21:43 +08:00
Dongjoon Hyun 4e7a29efdb [SPARK-21913][SQL][TEST] withDatabase` should drop database with CASCADE
## What changes were proposed in this pull request?

Currently, `withDatabase` fails if the database is not empty. It would be great if we drop cleanly with CASCADE.

## How was this patch tested?

This is a change on test util. Pass the existing Jenkins.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19125 from dongjoon-hyun/SPARK-21913.
2017-09-05 00:20:16 -07:00
Sean Owen ca59445adb [SPARK-21418][SQL] NoSuchElementException: None.get in DataSourceScanExec with sun.io.serialization.extendedDebugInfo=true
## What changes were proposed in this pull request?

If no SparkConf is available to Utils.redact, simply don't redact.

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #19123 from srowen/SPARK-21418.
2017-09-04 23:02:59 +02:00
Liang-Chi Hsieh 9f30d92803 [SPARK-21654][SQL] Complement SQL predicates expression description
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

Existing tests. And added predicate test.

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

Closes #18869 from viirya/SPARK-21654.
2017-09-03 21:55:18 -07:00
gatorsmile acb7fed237 [SPARK-21891][SQL] Add TBLPROPERTIES to DDL statement: CREATE TABLE USING
## What changes were proposed in this pull request?
Add `TBLPROPERTIES` to the DDL statement `CREATE TABLE USING`.

After this change, the DDL becomes
```
CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db_name.]table_name
USING table_provider
[OPTIONS table_property_list]
[PARTITIONED BY (col_name, col_name, ...)]
[CLUSTERED BY (col_name, col_name, ...)
 [SORTED BY (col_name [ASC|DESC], ...)]
 INTO num_buckets BUCKETS
]
[LOCATION path]
[COMMENT table_comment]
[TBLPROPERTIES (property_name=property_value, ...)]
[[AS] select_statement];
```

## How was this patch tested?
Add a few tests

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19100 from gatorsmile/addTablePropsToCreateTableUsing.
2017-09-02 14:53:41 -07:00
Sean Owen 12ab7f7e89 [SPARK-14280][BUILD][WIP] Update change-version.sh and pom.xml to add Scala 2.12 profiles and enable 2.12 compilation
…build; fix some things that will be warnings or errors in 2.12; restore Scala 2.12 profile infrastructure

## What changes were proposed in this pull request?

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

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

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

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

This change does _not_ fully enable a Scala 2.12 build:

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

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

## How was this patch tested?

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

Author: Sean Owen <sowen@cloudera.com>

Closes #18645 from srowen/SPARK-14280.
2017-09-01 19:21:21 +01:00
he.qiao 12f0d24225 [SPARK-21880][WEB UI] In the SQL table page, modify jobs trace information
## What changes were proposed in this pull request?
As shown below, for example, When the job 5 is running, It was a mistake to think that five jobs were running, So I think it would be more appropriate to change jobs to job id.
![image](https://user-images.githubusercontent.com/21355020/29909612-4dc85064-8e59-11e7-87cd-275a869243bb.png)

## How was this patch tested?
no need

Author: he.qiao <he.qiao17@zte.com.cn>

Closes #19093 from Geek-He/08_31_sqltable.
2017-09-01 10:47:11 -07:00
hyukjinkwon 5cd8ea99f0 [SPARK-21779][PYTHON] Simpler DataFrame.sample API in Python
## What changes were proposed in this pull request?

This PR make `DataFrame.sample(...)` can omit `withReplacement` defaulting `False`, consistently with equivalent Scala / Java API.

In short, the following examples are allowed:

```python
>>> df = spark.range(10)
>>> df.sample(0.5).count()
7
>>> df.sample(fraction=0.5).count()
3
>>> df.sample(0.5, seed=42).count()
5
>>> df.sample(fraction=0.5, seed=42).count()
5
```

In addition, this PR also adds some type checking logics as below:

```python
>>> df = spark.range(10)
>>> df.sample().count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [].
>>> df.sample(True).count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'bool'>].
>>> df.sample(42).count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'int'>].
>>> df.sample(fraction=False, seed="a").count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'bool'>, <type 'str'>].
>>> df.sample(seed=[1]).count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'list'>].
>>> df.sample(withReplacement="a", fraction=0.5, seed=1)
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'str'>, <type 'float'>, <type 'int'>].
```

## How was this patch tested?

Manually tested, unit tests added in doc tests and manually checked the built documentation for Python.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18999 from HyukjinKwon/SPARK-21779.
2017-09-01 13:01:23 +09:00
Bryan Cutler 501370d9d5 [SPARK-21583][HOTFIX] Removed intercept in test causing failures
Removing a check in the ColumnarBatchSuite that depended on a Java assertion.  This assertion is being compiled out in the Maven builds causing the test to fail.  This part of the test is not specifically from to the functionality that is being tested here.

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #19098 from BryanCutler/hotfix-ColumnarBatchSuite-assertion.
2017-08-31 11:32:10 -07:00
Jacek Laskowski 9696580c33 [SPARK-21886][SQL] Use SparkSession.internalCreateDataFrame to create…
… Dataset with LogicalRDD logical operator

## What changes were proposed in this pull request?

Reusing `SparkSession.internalCreateDataFrame` wherever possible (to cut dups)

## How was this patch tested?

Local build and waiting for Jenkins

Author: Jacek Laskowski <jacek@japila.pl>

Closes #19095 from jaceklaskowski/SPARK-21886-internalCreateDataFrame.
2017-08-31 09:44:29 -07:00
gatorsmile 19b0240d42 [SPARK-21878][SQL][TEST] Create SQLMetricsTestUtils
## What changes were proposed in this pull request?
Creates `SQLMetricsTestUtils` for the utility functions of both Hive-specific and the other SQLMetrics test cases.

Also, move two SQLMetrics test cases from sql/hive to sql/core.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19092 from gatorsmile/rewriteSQLMetrics.
2017-08-31 09:16:26 -07:00
Bryan Cutler 964b507c75 [SPARK-21583][SQL] Create a ColumnarBatch from ArrowColumnVectors
## What changes were proposed in this pull request?

This PR allows the creation of a `ColumnarBatch` from `ReadOnlyColumnVectors` where previously a columnar batch could only allocate vectors internally.  This is useful for using `ArrowColumnVectors` in a batch form to do row-based iteration.  Also added `ArrowConverter.fromPayloadIterator` which converts `ArrowPayload` iterator to `InternalRow` iterator and uses a `ColumnarBatch` internally.

## How was this patch tested?

Added a new unit test for creating a `ColumnarBatch` with `ReadOnlyColumnVectors` and a test to verify the roundtrip of rows -> ArrowPayload -> rows, using `toPayloadIterator` and `fromPayloadIterator`.

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #18787 from BryanCutler/arrow-ColumnarBatch-support-SPARK-21583.
2017-08-31 13:08:52 +09:00
Andrew Ash 313c6ca435 [SPARK-21875][BUILD] Fix Java style bugs
## What changes were proposed in this pull request?

Fix Java code style so `./dev/lint-java` succeeds

## How was this patch tested?

Run `./dev/lint-java`

Author: Andrew Ash <andrew@andrewash.com>

Closes #19088 from ash211/spark-21875-lint-java.
2017-08-31 09:26:11 +09:00
Dongjoon Hyun d8f4540863 [SPARK-21839][SQL] Support SQL config for ORC compression
## What changes were proposed in this pull request?

This PR aims to support `spark.sql.orc.compression.codec` like Parquet's `spark.sql.parquet.compression.codec`. Users can use SQLConf to control ORC compression, too.

## How was this patch tested?

Pass the Jenkins with new and updated test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19055 from dongjoon-hyun/SPARK-21839.
2017-08-31 08:16:58 +09:00
caoxuewen 235d28333c [MINOR][SQL][TEST] Test shuffle hash join while is not expected
## What changes were proposed in this pull request?

igore("shuffle hash join") is to shuffle hash join to test _case class ShuffledHashJoinExec_.
But when you 'ignore' -> 'test', the test is _case class BroadcastHashJoinExec_.

Before modified,  as a result of:canBroadcast is true.
Print information in _canBroadcast(plan: LogicalPlan)_
```
canBroadcast plan.stats.sizeInBytes:6710880
canBroadcast conf.autoBroadcastJoinThreshold:10000000
```

After modified, plan.stats.sizeInBytes is 11184808.
Print information in _canBuildLocalHashMap(plan: LogicalPlan)_
and _muchSmaller(a: LogicalPlan, b: LogicalPlan)_ :

```
canBuildLocalHashMap plan.stats.sizeInBytes:11184808
canBuildLocalHashMap conf.autoBroadcastJoinThreshold:10000000
canBuildLocalHashMap conf.numShufflePartitions:2
```
```
muchSmaller a.stats.sizeInBytes * 3:33554424
muchSmaller b.stats.sizeInBytes:33554432
```
## How was this patch tested?

existing test case.

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

Closes #19069 from heary-cao/shuffle_hash_join.
2017-08-30 10:10:24 -07:00
gatorsmile 32d6d9d720 Revert "[SPARK-21845][SQL] Make codegen fallback of expressions configurable"
This reverts commit 3d0e174244.
2017-08-30 09:08:40 -07:00
gatorsmile 3d0e174244 [SPARK-21845][SQL] Make codegen fallback of expressions configurable
## What changes were proposed in this pull request?
We should make codegen fallback of expressions configurable. So far, it is always on. We might hide it when our codegen have compilation bugs. Thus, we should also disable the codegen fallback when running test cases.

## How was this patch tested?
Added test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19062 from gatorsmile/fallbackCodegen.
2017-08-29 20:59:01 -07:00
Wenchen Fan 6327ea570b [SPARK-21255][SQL] simplify encoder for java enum
## What changes were proposed in this pull request?

This is a follow-up for https://github.com/apache/spark/pull/18488, to simplify the code.

The major change is, we should map java enum to string type, instead of a struct type with a single string field.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19066 from cloud-fan/fix.
2017-08-29 09:15:59 -07:00
Wang Gengliang 8fcbda9c93 [SPARK-21848][SQL] Add trait UserDefinedExpression to identify user-defined functions
## What changes were proposed in this pull request?

Add trait UserDefinedExpression to identify user-defined functions.
UDF can be expensive. In optimizer we may need to avoid executing UDF multiple times.
E.g.
```scala
table.select(UDF as 'a).select('a, ('a + 1) as 'b)
```
If UDF is expensive in this case, optimizer should not collapse the project to
```scala
table.select(UDF as 'a, (UDF+1) as 'b)
```

Currently UDF classes like PythonUDF, HiveGenericUDF are not defined in catalyst.
This PR is to add a new trait to make it easier to identify user-defined functions.

## How was this patch tested?

Unit test

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #19064 from gengliangwang/UDFType.
2017-08-29 09:08:59 -07:00
Takuya UESHIN 32fa0b8141 [SPARK-21781][SQL] Modify DataSourceScanExec to use concrete ColumnVector type.
## What changes were proposed in this pull request?

As mentioned at https://github.com/apache/spark/pull/18680#issuecomment-316820409, when we have more `ColumnVector` implementations, it might (or might not) have huge performance implications because it might disable inlining, or force virtual dispatches.

As for read path, one of the major paths is the one generated by `ColumnBatchScan`. Currently it refers `ColumnVector` so the penalty will be bigger as we have more classes, but we can know the concrete type from its usage, e.g. vectorized Parquet reader uses `OnHeapColumnVector`. We can use the concrete type in the generated code directly to avoid the penalty.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #18989 from ueshin/issues/SPARK-21781.
2017-08-29 20:16:45 +08:00
iamhumanbeing 07142cf6dc [SPARK-21843] testNameNote should be "(minNumPostShufflePartitions: 5)"
Signed-off-by: iamhumanbeing <iamhumanbeinggmail.com>

## What changes were proposed in this pull request?

testNameNote = "(minNumPostShufflePartitions: 3) is not correct.
it should be "(minNumPostShufflePartitions: " + numPartitions + ")" in ExchangeCoordinatorSuite

## How was this patch tested?

unit tests

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

Author: iamhumanbeing <iamhumanbeing@gmail.com>

Closes #19058 from iamhumanbeing/testnote.
2017-08-27 08:23:57 +01:00
Sean Owen 1a598d717c [SPARK-21837][SQL][TESTS] UserDefinedTypeSuite Local UDTs not actually testing what it intends
## What changes were proposed in this pull request?

Adjust Local UDTs test to assert about results, and fix index of vector column. See JIRA for details.

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #19053 from srowen/SPARK-21837.
2017-08-25 13:29:40 -07:00
vinodkc 51620e288b [SPARK-21756][SQL] Add JSON option to allow unquoted control characters
## What changes were proposed in this pull request?

This patch adds allowUnquotedControlChars option in JSON data source to allow JSON Strings to contain unquoted control characters (ASCII characters with value less than 32, including tab and line feed characters)

## How was this patch tested?
Add new test cases

Author: vinodkc <vinod.kc.in@gmail.com>

Closes #19008 from vinodkc/br_fix_SPARK-21756.
2017-08-25 10:18:03 -07:00
Sean Owen de7af295c2 [MINOR][BUILD] Fix build warnings and Java lint errors
## What changes were proposed in this pull request?

Fix build warnings and Java lint errors. This just helps a bit in evaluating (new) warnings in another PR I have open.

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #19051 from srowen/JavaWarnings.
2017-08-25 16:07:13 +01:00
mike 7d16776d28 [SPARK-21255][SQL][WIP] Fixed NPE when creating encoder for enum
## What changes were proposed in this pull request?

Fixed NPE when creating encoder for enum.

When you try to create an encoder for Enum type (or bean with enum property) via Encoders.bean(...), it fails with NullPointerException at TypeToken:495.
I did a little research and it turns out, that in JavaTypeInference following code
```
  def getJavaBeanReadableProperties(beanClass: Class[_]): Array[PropertyDescriptor] = {
    val beanInfo = Introspector.getBeanInfo(beanClass)
    beanInfo.getPropertyDescriptors.filterNot(_.getName == "class")
      .filter(_.getReadMethod != null)
  }
```
filters out properties named "class", because we wouldn't want to serialize that. But enum types have another property of type Class named "declaringClass", which we are trying to inspect recursively. Eventually we try to inspect ClassLoader class, which has property "defaultAssertionStatus" with no read method, which leads to NPE at TypeToken:495.

I added property name "declaringClass" to filtering to resolve this.

## How was this patch tested?
Unit test in JavaDatasetSuite which creates an encoder for enum

Author: mike <mike0sv@gmail.com>
Author: Mikhail Sveshnikov <mike0sv@gmail.com>

Closes #18488 from mike0sv/enum-support.
2017-08-25 07:22:34 +01:00
Herman van Hovell 05af2de0fd [SPARK-21830][SQL] Bump ANTLR version and fix a few issues.
## What changes were proposed in this pull request?
This PR bumps the ANTLR version to 4.7, and fixes a number of small parser related issues uncovered by the bump.

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

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

## How was this patch tested?
Existing tests.

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

Closes #19042 from hvanhovell/SPARK-21830.
2017-08-24 16:33:55 -07:00
Shixiong Zhu d3abb36990 [SPARK-21788][SS] Handle more exceptions when stopping a streaming query
## What changes were proposed in this pull request?

Add more cases we should view as a normal query stop rather than a failure.

## How was this patch tested?

The new unit tests.

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #18997 from zsxwing/SPARK-21788.
2017-08-24 10:23:59 -07:00
Wenchen Fan 2dd37d827f [SPARK-21826][SQL] outer broadcast hash join should not throw NPE
## What changes were proposed in this pull request?

This is a bug introduced by https://github.com/apache/spark/pull/11274/files#diff-7adb688cbfa583b5711801f196a074bbL274 .

Non-equal join condition should only be applied when the equal-join condition matches.

## How was this patch tested?

regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19036 from cloud-fan/bug.
2017-08-24 16:44:12 +02:00
Liang-Chi Hsieh 183d4cb71f [SPARK-21759][SQL] In.checkInputDataTypes should not wrongly report unresolved plans for IN correlated subquery
## What changes were proposed in this pull request?

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

For a correlated IN query looks like:

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

The query plan might look like:

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

After `PullupCorrelatedPredicates`, it produces query plan like:

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

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

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

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

## How was this patch tested?

Added test.

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

Closes #18968 from viirya/SPARK-21759.
2017-08-24 21:46:58 +08:00
Takuya UESHIN 9e33954ddf [SPARK-21745][SQL] Refactor ColumnVector hierarchy to make ColumnVector read-only and to introduce WritableColumnVector.
## What changes were proposed in this pull request?

This is a refactoring of `ColumnVector` hierarchy and related classes.

1. make `ColumnVector` read-only
2. introduce `WritableColumnVector` with write interface
3. remove `ReadOnlyColumnVector`

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #18958 from ueshin/issues/SPARK-21745.
2017-08-24 21:13:44 +08:00
lufei 846bc61cf5 [MINOR][SQL] The comment of Class ExchangeCoordinator exist a typing and context error
## What changes were proposed in this pull request?

The given example in the comment of Class ExchangeCoordinator is exist four post-shuffle partitions,but the current comment is “three”.

## How was this patch tested?

Author: lufei <lu.fei80@zte.com.cn>

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

## How was this patch tested?

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

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

Closes #18973 from joseph-torres/SPARK-21765.
2017-08-22 19:07:43 -07:00
gatorsmile 01a8e46278 [SPARK-21769][SQL] Add a table-specific option for always respecting schemas inferred/controlled by Spark SQL
## What changes were proposed in this pull request?
For Hive-serde tables, we always respect the schema stored in Hive metastore, because the schema could be altered by the other engines that share the same metastore. Thus, we always trust the metastore-controlled schema for Hive-serde tables when the schemas are different (without considering the nullability and cases). However, in some scenarios, Hive metastore also could INCORRECTLY overwrite the schemas when the serde and Hive metastore built-in serde are different.

The proposed solution is to introduce a table-specific option for such scenarios. For a specific table, users can make Spark always respect Spark-inferred/controlled schema instead of trusting metastore-controlled schema. By default, we trust Hive metastore-controlled schema.

## How was this patch tested?
Added a cross-version test case

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19003 from gatorsmile/respectSparkSchema.
2017-08-22 13:12:59 -07:00