Commit graph

5802 commits

Author SHA1 Message Date
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
hyukjinkwon 9d48bd0b34 [SPARK-22093][TESTS] Fixes assume in UtilsSuite and HiveDDLSuite
## What changes were proposed in this pull request?

This PR proposes to remove `assume` in `Utils.resolveURIs` and replace `assume` to `assert` in `Utils.resolveURI` in the test cases in `UtilsSuite`.

It looks `Utils.resolveURIs` supports multiple but also single paths as input. So, it looks not meaningful to check if the input has `,`.

For the test for `Utils.resolveURI`, I replaced it to `assert` because it looks taking single path and in order to prevent future mistakes when adding more tests here.

For `assume` in `HiveDDLSuite`, it looks it should be `assert` to test at the last
## How was this patch tested?

Fixed unit tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #19332 from HyukjinKwon/SPARK-22093.
2017-09-24 17:11:29 +09: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
Kevin Yu 4a8c9e29bc [SPARK-22110][SQL][DOCUMENTATION] Add usage and improve documentation with arguments and examples for trim function
## What changes were proposed in this pull request?

This PR proposes to enhance the documentation for `trim` functions in the function description session.

- Add more `usage`, `arguments` and `examples` for the trim function
- Adjust space in the `usage` session

After the changes, the trim function documentation will look like this:

- `trim`

```trim(str) - Removes the leading and trailing space characters from str.

trim(BOTH trimStr FROM str) - Remove the leading and trailing trimStr characters from str

trim(LEADING trimStr FROM str) - Remove the leading trimStr characters from str

trim(TRAILING trimStr FROM str) - Remove the trailing trimStr characters from str

Arguments:

str - a string expression
trimStr - the trim string characters to trim, the default value is a single space
BOTH, FROM - these are keywords to specify trimming string characters from both ends of the string
LEADING, FROM - these are keywords to specify trimming string characters from the left end of the string
TRAILING, FROM - these are keywords to specify trimming string characters from the right end of the string
Examples:

> SELECT trim('    SparkSQL   ');
 SparkSQL
> SELECT trim('SL', 'SSparkSQLS');
 parkSQ
> SELECT trim(BOTH 'SL' FROM 'SSparkSQLS');
 parkSQ
> SELECT trim(LEADING 'SL' FROM 'SSparkSQLS');
 parkSQLS
> SELECT trim(TRAILING 'SL' FROM 'SSparkSQLS');
 SSparkSQ
```

- `ltrim`

```ltrim

ltrim(str) - Removes the leading space characters from str.

ltrim(trimStr, str) - Removes the leading string contains the characters from the trim string

Arguments:

str - a string expression
trimStr - the trim string characters to trim, the default value is a single space
Examples:

> SELECT ltrim('    SparkSQL   ');
 SparkSQL
> SELECT ltrim('Sp', 'SSparkSQLS');
 arkSQLS
```

- `rtrim`
```rtrim

rtrim(str) - Removes the trailing space characters from str.

rtrim(trimStr, str) - Removes the trailing string which contains the characters from the trim string from the str

Arguments:

str - a string expression
trimStr - the trim string characters to trim, the default value is a single space
Examples:

> SELECT rtrim('    SparkSQL   ');
 SparkSQL
> SELECT rtrim('LQSa', 'SSparkSQLS');
 SSpark
```

This is the trim characters function jira: [trim function](https://issues.apache.org/jira/browse/SPARK-14878)

## How was this patch tested?

Manually tested
```
spark-sql> describe function extended trim;
17/09/22 17:03:04 INFO CodeGenerator: Code generated in 153.026533 ms
Function: trim
Class: org.apache.spark.sql.catalyst.expressions.StringTrim
Usage:
    trim(str) - Removes the leading and trailing space characters from `str`.

    trim(BOTH trimStr FROM str) - Remove the leading and trailing `trimStr` characters from `str`

    trim(LEADING trimStr FROM str) - Remove the leading `trimStr` characters from `str`

    trim(TRAILING trimStr FROM str) - Remove the trailing `trimStr` characters from `str`

Extended Usage:
    Arguments:
      * str - a string expression
      * trimStr - the trim string characters to trim, the default value is a single space
      * BOTH, FROM - these are keywords to specify trimming string characters from both ends of
          the string
      * LEADING, FROM - these are keywords to specify trimming string characters from the left
          end of the string
      * TRAILING, FROM - these are keywords to specify trimming string characters from the right
          end of the string

    Examples:
      > SELECT trim('    SparkSQL   ');
       SparkSQL
      > SELECT trim('SL', 'SSparkSQLS');
       parkSQ
      > SELECT trim(BOTH 'SL' FROM 'SSparkSQLS');
       parkSQ
      > SELECT trim(LEADING 'SL' FROM 'SSparkSQLS');
       parkSQLS
      > SELECT trim(TRAILING 'SL' FROM 'SSparkSQLS');
       SSparkSQ
  ```
```
spark-sql> describe function extended ltrim;
Function: ltrim
Class: org.apache.spark.sql.catalyst.expressions.StringTrimLeft
Usage:
    ltrim(str) - Removes the leading space characters from `str`.

    ltrim(trimStr, str) - Removes the leading string contains the characters from the trim string

Extended Usage:
    Arguments:
      * str - a string expression
      * trimStr - the trim string characters to trim, the default value is a single space

    Examples:
      > SELECT ltrim('    SparkSQL   ');
       SparkSQL
      > SELECT ltrim('Sp', 'SSparkSQLS');
       arkSQLS

```

```
spark-sql> describe function extended rtrim;
Function: rtrim
Class: org.apache.spark.sql.catalyst.expressions.StringTrimRight
Usage:
    rtrim(str) - Removes the trailing space characters from `str`.

    rtrim(trimStr, str) - Removes the trailing string which contains the characters from the trim string from the `str`

Extended Usage:
    Arguments:
      * str - a string expression
      * trimStr - the trim string characters to trim, the default value is a single space

    Examples:
      > SELECT rtrim('    SparkSQL   ');
       SparkSQL
      > SELECT rtrim('LQSa', 'SSparkSQLS');
       SSpark

```

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

Closes #19329 from kevinyu98/spark-14878-5.
2017-09-23 10:27:40 -07:00
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
Liang-Chi Hsieh 9cac249fd5 [SPARK-22088][SQL] Incorrect scalastyle comment causes wrong styles in stringExpressions
## What changes were proposed in this pull request?

There is an incorrect `scalastyle:on` comment in `stringExpressions.scala` and causes the line size limit check ineffective in the file. There are many lines of code and comment which are more than 100 chars.

## How was this patch tested?

Code style change only.

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

Closes #19305 from viirya/fix-wrong-style.
2017-09-21 11:51:00 -07:00
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
Liang-Chi Hsieh 1270e71753 [SPARK-22086][DOCS] Add expression description for CASE WHEN
## What changes were proposed in this pull request?

In SQL conditional expressions, only CASE WHEN lacks for expression description. This patch fills the gap.

## How was this patch tested?

Only documentation change.

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

Closes #19304 from viirya/casewhen-doc.
2017-09-21 22:45:06 +09:00
Zhenhua Wang 1d1a09be9f [SPARK-17997][SQL] Add an aggregation function for counting distinct values for multiple intervals
## What changes were proposed in this pull request?

This work is a part of [SPARK-17074](https://issues.apache.org/jira/browse/SPARK-17074) to compute equi-height histograms. Equi-height histogram is an array of bins. A bin consists of two endpoints which form an interval of values and the ndv in that interval.

This PR creates a new aggregate function, given an array of endpoints, counting distinct values (ndv) in intervals among those endpoints.

This PR also refactors `HyperLogLogPlusPlus` by extracting a helper class `HyperLogLogPlusPlusHelper`, where the underlying HLLPP algorithm locates.

## How was this patch tested?

Add new test cases.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #15544 from wzhfy/countIntervals.
2017-09-21 21:43:02 +08:00
Wenchen Fan 352bea5457 [SPARK-22076][SQL][FOLLOWUP] Expand.projections should not be a Stream
## What changes were proposed in this pull request?

This a follow-up of https://github.com/apache/spark/pull/19289 , we missed another place: `rollup`. `Seq.init.toSeq` also returns a `Stream`, we should fix it too.

## How was this patch tested?

manually

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19298 from cloud-fan/bug.
2017-09-20 21:13:46 -07:00
Wenchen Fan ce6a71e013 [SPARK-22076][SQL] Expand.projections should not be a Stream
## What changes were proposed in this pull request?

Spark with Scala 2.10 fails with a group by cube:
```
spark.range(1).select($"id" as "a", $"id" as "b").write.partitionBy("a").mode("overwrite").saveAsTable("rollup_bug")
spark.sql("select 1 from rollup_bug group by rollup ()").show
```

It can be traced back to https://github.com/apache/spark/pull/15484 , which made `Expand.projections` a lazy `Stream` for group by cube.

In scala 2.10 `Stream` captures a lot of stuff, and in this case it captures the entire query plan which has some un-serializable parts.

This change is also good for master branch, to reduce the serialized size of `Expand.projections`.

## How was this patch tested?

manually verified with Spark with Scala 2.10.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19289 from cloud-fan/bug.
2017-09-20 09:00:43 -07:00
Sean Owen e17901d6df [SPARK-22049][DOCS] Confusing behavior of from_utc_timestamp and to_utc_timestamp
## What changes were proposed in this pull request?

Clarify behavior of to_utc_timestamp/from_utc_timestamp with an example

## How was this patch tested?

Doc only change / existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #19276 from srowen/SPARK-22049.
2017-09-20 20:47:17 +09:00
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
Kent Yao 581200af71 [SPARK-21428][SQL][FOLLOWUP] CliSessionState should point to the actual metastore not a dummy one
## What changes were proposed in this pull request?

While running bin/spark-sql, we will reuse cliSessionState, but the Hive configurations generated here just points to a dummy meta store which actually should be the real one. And the warehouse is determined later in SharedState, HiveClient should respect this config changing in this case too.

## How was this patch tested?
existing ut

cc cloud-fan jiangxb1987

Author: Kent Yao <yaooqinn@hotmail.com>

Closes #19068 from yaooqinn/SPARK-21428-FOLLOWUP.
2017-09-19 19:35:36 +08: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
Armin 7c92351f43 [MINOR][CORE] Cleanup dead code and duplication in Mem. Management
## What changes were proposed in this pull request?

* Removed the method `org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter#alignToWords`.
It became unused as a result of 85b0a15754
(SPARK-15962) introducing word alignment for unsafe arrays.
* Cleaned up duplicate code in memory management and unsafe sorters
  * The change extracting the exception paths is more than just cosmetics since it def. reduces the size the affected methods compile to

## How was this patch tested?

* Build still passes after removing the method, grepping the codebase for `alignToWords` shows no reference to it anywhere either.
* Dried up code is covered by existing tests.

Author: Armin <me@obrown.io>

Closes #19254 from original-brownbear/cleanup-mem-consumer.
2017-09-19 10:06:32 +01:00
Wenchen Fan 10f45b3c84 [SPARK-22047][FLAKY TEST] HiveExternalCatalogVersionsSuite
## What changes were proposed in this pull request?

This PR tries to download Spark for each test run, to make sure each test run is absolutely isolated.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19265 from cloud-fan/test.
2017-09-19 11:53:50 +08: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
Wenchen Fan 894a7561de [SPARK-22047][TEST] ignore HiveExternalCatalogVersionsSuite
## What changes were proposed in this pull request?

As reported in https://issues.apache.org/jira/browse/SPARK-22047 , HiveExternalCatalogVersionsSuite is failing frequently, let's disable this test suite to unblock other PRs, I'm looking into the root cause.

## How was this patch tested?
N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19264 from cloud-fan/test.
2017-09-18 16:42:08 +08: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
Tathagata Das 88661747f5 [SPARK-22018][SQL] Preserve top-level alias metadata when collapsing projects
## What changes were proposed in this pull request?
If there are two projects like as follows.
```
Project [a_with_metadata#27 AS b#26]
+- Project [a#0 AS a_with_metadata#27]
   +- LocalRelation <empty>, [a#0, b#1]
```
Child Project has an output column with a metadata in it, and the parent Project has an alias that implicitly forwards the metadata. So this metadata is visible for higher operators. Upon applying CollapseProject optimizer rule, the metadata is not preserved.
```
Project [a#0 AS b#26]
+- LocalRelation <empty>, [a#0, b#1]
```
This is incorrect, as downstream operators that expect certain metadata (e.g. watermark in structured streaming) to identify certain fields will fail to do so. This PR fixes it by preserving the metadata of top-level aliases.

## How was this patch tested?
New unit test

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

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

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

cc viirya HyukjinKwon

Author: goldmedal <liugs963@gmail.com>

Closes #19223 from goldmedal/SPARK-21513-fp-PySaprkAndSparkR.
2017-09-15 11:53:10 +09:00
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
Dilip Biswal dcbb229433 [MINOR][SQL] Only populate type metadata for required types such as CHAR/VARCHAR.
## What changes were proposed in this pull request?
When reading column descriptions from hive catalog, we currently populate the metadata for all types to record the raw hive type string. In terms of processing , we need this additional metadata information for CHAR/VARCHAR types or complex type containing the CHAR/VARCHAR types.

Its a minor cleanup. I haven't created a JIRA for it.

## How was this patch tested?
Test added in HiveMetastoreCatalogSuite

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

Closes #19215 from dilipbiswal/column_metadata.
2017-09-13 22:45:44 -07: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
Jane Wang 8c7e19a37d [SPARK-4131] Merge HiveTmpFile.scala to SaveAsHiveFile.scala
## What changes were proposed in this pull request?

The code is already merged to master:
https://github.com/apache/spark/pull/18975

This is a following up PR to merge HiveTmpFile.scala to SaveAsHiveFile.

## How was this patch tested?

Build successfully

Author: Jane Wang <janewang@fb.com>

Closes #19221 from janewangfb/merge_savehivefile_hivetmpfile.
2017-09-13 15:12:36 -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