Commit graph

19543 commits

Author SHA1 Message Date
Dongjoon Hyun bfc8c79c8d [SPARK-20566][SQL] ColumnVector should support appendFloats for array
## What changes were proposed in this pull request?

This PR aims to add a missing `appendFloats` API for array into **ColumnVector** class. For double type, there is `appendDoubles` for array [here](https://github.com/apache/spark/blob/master/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java#L818-L824).

## How was this patch tested?

Pass the Jenkins with a newly added test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #17836 from dongjoon-hyun/SPARK-20566.
2017-05-04 21:04:15 +08:00
Yanbo Liang c5dceb8c65 [SPARK-20047][FOLLOWUP][ML] Constrained Logistic Regression follow up
## What changes were proposed in this pull request?
Address some minor comments for #17715:
* Put bound-constrained optimization params under expertParams.
* Update some docs.

## How was this patch tested?
Existing tests.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #17829 from yanboliang/spark-20047-followup.
2017-05-04 17:56:43 +08:00
Felix Cheung 57b64703e6 [SPARK-20571][SPARKR][SS] Flaky Structured Streaming tests
## What changes were proposed in this pull request?

Make tests more reliable by having it till processed.
Increasing timeout value might help but ultimately the flakiness from processing delay when Jenkins is hard to account for. This isn't an actual public API supported

## How was this patch tested?
unit tests

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #17857 from felixcheung/rsstestrelia.
2017-05-04 01:54:59 -07:00
zero323 f21897fc15 [SPARK-20544][SPARKR] R wrapper for input_file_name
## What changes were proposed in this pull request?

Adds wrapper for `o.a.s.sql.functions.input_file_name`

## How was this patch tested?

Existing unit tests, additional unit tests, `check-cran.sh`.

Author: zero323 <zero323@users.noreply.github.com>

Closes #17818 from zero323/SPARK-20544.
2017-05-04 01:51:37 -07:00
zero323 9c36aa2791 [SPARK-20585][SPARKR] R generic hint support
## What changes were proposed in this pull request?

Adds support for generic hints on `SparkDataFrame`

## How was this patch tested?

Unit tests, `check-cran.sh`

Author: zero323 <zero323@users.noreply.github.com>

Closes #17851 from zero323/SPARK-20585.
2017-05-04 01:41:36 -07:00
Felix Cheung b8302ccd02 [SPARK-20015][SPARKR][SS][DOC][EXAMPLE] Document R Structured Streaming (experimental) in R vignettes and R & SS programming guide, R example
## What changes were proposed in this pull request?

Add
- R vignettes
- R programming guide
- SS programming guide
- R example

Also disable spark.als in vignettes for now since it's failing (SPARK-20402)

## How was this patch tested?

manually

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #17814 from felixcheung/rdocss.
2017-05-04 00:27:10 -07:00
Felix Cheung fc472bddd1 [SPARK-20543][SPARKR] skip tests when running on CRAN
## What changes were proposed in this pull request?

General rule on skip or not:
skip if
- RDD tests
- tests could run long or complicated (streaming, hivecontext)
- tests on error conditions
- tests won't likely change/break

## How was this patch tested?

unit tests, `R CMD check --as-cran`, `R CMD check`

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #17817 from felixcheung/rskiptest.
2017-05-03 21:40:18 -07:00
zero323 02bbe73118 [SPARK-20584][PYSPARK][SQL] Python generic hint support
## What changes were proposed in this pull request?

Adds `hint` method to PySpark `DataFrame`.

## How was this patch tested?

Unit tests, doctests.

Author: zero323 <zero323@users.noreply.github.com>

Closes #17850 from zero323/SPARK-20584.
2017-05-03 19:15:28 -07:00
hyukjinkwon 13eb37c860 [MINOR][SQL] Fix the test title from =!= to <=>, remove a duplicated test and add a test for =!=
## What changes were proposed in this pull request?

This PR proposes three things as below:

- This test looks not testing `<=>` and identical with the test above, `===`. So, it removes the test.

  ```diff
  -   test("<=>") {
  -     checkAnswer(
  -      testData2.filter($"a" === 1),
  -      testData2.collect().toSeq.filter(r => r.getInt(0) == 1))
  -
  -    checkAnswer(
  -      testData2.filter($"a" === $"b"),
  -      testData2.collect().toSeq.filter(r => r.getInt(0) == r.getInt(1)))
  -   }
  ```

- Replace the test title from `=!=` to `<=>`. It looks the test actually testing `<=>`.

  ```diff
  +  private lazy val nullData = Seq(
  +    (Some(1), Some(1)), (Some(1), Some(2)), (Some(1), None), (None, None)).toDF("a", "b")
  +
    ...
  -  test("=!=") {
  +  test("<=>") {
  -    val nullData = spark.createDataFrame(sparkContext.parallelize(
  -      Row(1, 1) ::
  -      Row(1, 2) ::
  -      Row(1, null) ::
  -      Row(null, null) :: Nil),
  -      StructType(Seq(StructField("a", IntegerType), StructField("b", IntegerType))))
  -
         checkAnswer(
           nullData.filter($"b" <=> 1),
    ...
  ```

- Add the tests for `=!=` which looks not existing.

  ```diff
  +  test("=!=") {
  +    checkAnswer(
  +      nullData.filter($"b" =!= 1),
  +      Row(1, 2) :: Nil)
  +
  +    checkAnswer(nullData.filter($"b" =!= null), Nil)
  +
  +    checkAnswer(
  +      nullData.filter($"a" =!= $"b"),
  +      Row(1, 2) :: Nil)
  +  }
  ```

## How was this patch tested?

Manually running the tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17842 from HyukjinKwon/minor-test-fix.
2017-05-03 13:08:25 -07:00
Liwei Lin 6b9e49d12f [SPARK-19965][SS] DataFrame batch reader may fail to infer partitions when reading FileStreamSink's output
## The Problem

Right now DataFrame batch reader may fail to infer partitions when reading FileStreamSink's output:

```
[info] - partitioned writing and batch reading with 'basePath' *** FAILED *** (3 seconds, 928 milliseconds)
[info]   java.lang.AssertionError: assertion failed: Conflicting directory structures detected. Suspicious paths:
[info] 	***/stream.output-65e3fa45-595a-4d29-b3df-4c001e321637
[info] 	***/stream.output-65e3fa45-595a-4d29-b3df-4c001e321637/_spark_metadata
[info]
[info] If provided paths are partition directories, please set "basePath" in the options of the data source to specify the root directory of the table. If there are multiple root directories, please load them separately and then union them.
[info]   at scala.Predef$.assert(Predef.scala:170)
[info]   at org.apache.spark.sql.execution.datasources.PartitioningUtils$.parsePartitions(PartitioningUtils.scala:133)
[info]   at org.apache.spark.sql.execution.datasources.PartitioningUtils$.parsePartitions(PartitioningUtils.scala:98)
[info]   at org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex.inferPartitioning(PartitioningAwareFileIndex.scala:156)
[info]   at org.apache.spark.sql.execution.datasources.InMemoryFileIndex.partitionSpec(InMemoryFileIndex.scala:54)
[info]   at org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex.partitionSchema(PartitioningAwareFileIndex.scala:55)
[info]   at org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:133)
[info]   at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:361)
[info]   at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:160)
[info]   at org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:536)
[info]   at org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:520)
[info]   at org.apache.spark.sql.streaming.FileStreamSinkSuite$$anonfun$8.apply$mcV$sp(FileStreamSinkSuite.scala:292)
[info]   at org.apache.spark.sql.streaming.FileStreamSinkSuite$$anonfun$8.apply(FileStreamSinkSuite.scala:268)
[info]   at org.apache.spark.sql.streaming.FileStreamSinkSuite$$anonfun$8.apply(FileStreamSinkSuite.scala:268)
```

## What changes were proposed in this pull request?

This patch alters `InMemoryFileIndex` to filter out these `basePath`s whose ancestor is the streaming metadata dir (`_spark_metadata`). E.g., the following and other similar dir or files will be filtered out:
- (introduced by globbing `basePath/*`)
   - `basePath/_spark_metadata`
- (introduced by globbing `basePath/*/*`)
   - `basePath/_spark_metadata/0`
   - `basePath/_spark_metadata/1`
   - ...

## How was this patch tested?

Added unit tests

Author: Liwei Lin <lwlin7@gmail.com>

Closes #17346 from lw-lin/filter-metadata.
2017-05-03 11:10:24 -07:00
Reynold Xin 527fc5d0c9 [SPARK-20576][SQL] Support generic hint function in Dataset/DataFrame
## What changes were proposed in this pull request?
We allow users to specify hints (currently only "broadcast" is supported) in SQL and DataFrame. However, while SQL has a standard hint format (/*+ ... */), DataFrame doesn't have one and sometimes users are confused that they can't find how to apply a broadcast hint. This ticket adds a generic hint function on DataFrame that allows using the same hint on DataFrames as well as SQL.

As an example, after this patch, the following will apply a broadcast hint on a DataFrame using the new hint function:

```
df1.join(df2.hint("broadcast"))
```

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

Author: Reynold Xin <rxin@databricks.com>

Closes #17839 from rxin/SPARK-20576.
2017-05-03 09:22:25 -07:00
Liwei Lin 27f543b15f [SPARK-20441][SPARK-20432][SS] Within the same streaming query, one StreamingRelation should only be transformed to one StreamingExecutionRelation
## What changes were proposed in this pull request?

Within the same streaming query, when one `StreamingRelation` is referred multiple times – e.g. `df.union(df)` – we should transform it only to one `StreamingExecutionRelation`, instead of two or more different `StreamingExecutionRelation`s (each of which would have a separate set of source, source logs, ...).

## How was this patch tested?

Added two test cases, each of which would fail without this patch.

Author: Liwei Lin <lwlin7@gmail.com>

Closes #17735 from lw-lin/SPARK-20441.
2017-05-03 08:55:02 -07:00
Yan Facai (颜发才) 7f96f2d7f2 [SPARK-16957][MLLIB] Use midpoints for split values.
## What changes were proposed in this pull request?

Use midpoints for split values now, and maybe later to make it weighted.

## How was this patch tested?

+ [x] add unit test.
+ [x] revise Split's unit test.

Author: Yan Facai (颜发才) <facai.yan@gmail.com>
Author: 颜发才(Yan Facai) <facai.yan@gmail.com>

Closes #17556 from facaiy/ENH/decision_tree_overflow_and_precision_in_aggregation.
2017-05-03 10:54:40 +01:00
Sean Owen 16fab6b0ef [SPARK-20523][BUILD] Clean up build warnings for 2.2.0 release
## What changes were proposed in this pull request?

Fix build warnings primarily related to Breeze 0.13 operator changes, Java style problems

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #17803 from srowen/SPARK-20523.
2017-05-03 10:18:35 +01:00
MechCoder db2fb84b4a [SPARK-6227][MLLIB][PYSPARK] Implement PySpark wrappers for SVD and PCA (v2)
Add PCA and SVD to PySpark's wrappers for `RowMatrix` and `IndexedRowMatrix` (SVD only).

Based on #7963, updated.

## How was this patch tested?

New doc tests and unit tests. Ran all examples locally.

Author: MechCoder <manojkumarsivaraj334@gmail.com>
Author: Nick Pentreath <nickp@za.ibm.com>

Closes #17621 from MLnick/SPARK-6227-pyspark-svd-pca.
2017-05-03 10:58:05 +02:00
Michael Armbrust 6235132a8c [SPARK-20567] Lazily bind in GenerateExec
It is not valid to eagerly bind with the child's output as this causes failures when we attempt to canonicalize the plan (replacing the attribute references with dummies).

Author: Michael Armbrust <michael@databricks.com>

Closes #17838 from marmbrus/fixBindExplode.
2017-05-02 22:44:27 -07:00
Wenchen Fan b946f3160e [SPARK-20558][CORE] clear InheritableThreadLocal variables in SparkContext when stopping it
## What changes were proposed in this pull request?

To better understand this problem, let's take a look at an example first:
```
object Main {
  def main(args: Array[String]): Unit = {
    var t = new Test
    new Thread(new Runnable {
      override def run() = {}
    }).start()
    println("first thread finished")

    t.a = null
    t = new Test
    new Thread(new Runnable {
      override def run() = {}
    }).start()
  }

}

class Test {
  var a = new InheritableThreadLocal[String] {
    override protected def childValue(parent: String): String = {
      println("parent value is: " + parent)
      parent
    }
  }
  a.set("hello")
}
```
The result is:
```
parent value is: hello
first thread finished
parent value is: hello
parent value is: hello
```

Once an `InheritableThreadLocal` has been set value, child threads will inherit its value as long as it has not been GCed, so setting the variable which holds the `InheritableThreadLocal` to `null` doesn't work as we expected.

In `SparkContext`, we have an `InheritableThreadLocal` for local properties, we should clear it when stopping `SparkContext`, or all the future child threads will still inherit it and copy the properties and waste memory.

This is the root cause of https://issues.apache.org/jira/browse/SPARK-20548 , which creates/stops `SparkContext` many times and finally have a lot of `InheritableThreadLocal` alive, and cause OOM when starting new threads in the internal thread pools.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17833 from cloud-fan/core.
2017-05-03 10:08:46 +08:00
Marcelo Vanzin ef3df9125a [SPARK-20421][CORE] Add a missing deprecation tag.
In the previous patch I deprecated StorageStatus, but not the
method in SparkContext that exposes that class publicly. So deprecate
the method too.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #17824 from vanzin/SPARK-20421.
2017-05-02 14:30:06 -07:00
Felix Cheung 13f47dc503 [SPARK-20490][SPARKR][DOC] add family tag for not function
## What changes were proposed in this pull request?

doc only

## How was this patch tested?

manual

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #17828 from felixcheung/rnotfamily.
2017-05-02 09:37:01 -07:00
Xiao Li b1e639ab09 [SPARK-19235][SQL][TEST][FOLLOW-UP] Enable Test Cases in DDLSuite with Hive Metastore
### What changes were proposed in this pull request?
This is a follow-up of enabling test cases in DDLSuite with Hive Metastore. It consists of the following remaining tasks:
- Run all the `alter table` and `drop table` DDL tests against data source tables when using Hive metastore.
- Do not run any `alter table` and `drop table` DDL test against Hive serde tables when using InMemoryCatalog.
- Reenable `alter table: set serde partition` and `alter table: set serde` tests for Hive serde tables.

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17524 from gatorsmile/cleanupDDLSuite.
2017-05-02 16:49:24 +08:00
Nick Pentreath e300a5a145 [SPARK-20300][ML][PYSPARK] Python API for ALSModel.recommendForAllUsers,Items
Add Python API for `ALSModel` methods `recommendForAllUsers`, `recommendForAllItems`

## How was this patch tested?

New doc tests.

Author: Nick Pentreath <nickp@za.ibm.com>

Closes #17622 from MLnick/SPARK-20300-pyspark-recall.
2017-05-02 10:49:13 +02:00
Burak Yavuz 86174ea89b [SPARK-20549] java.io.CharConversionException: Invalid UTF-32' in JsonToStructs
## What changes were proposed in this pull request?

A fix for the same problem was made in #17693 but ignored `JsonToStructs`. This PR uses the same fix for `JsonToStructs`.

## How was this patch tested?

Regression test

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #17826 from brkyvz/SPARK-20549.
2017-05-02 14:08:16 +08:00
Kazuaki Ishizaki afb21bf22a [SPARK-20537][CORE] Fixing OffHeapColumnVector reallocation
## What changes were proposed in this pull request?

As #17773 revealed `OnHeapColumnVector` may copy a part of the original storage.

`OffHeapColumnVector` reallocation also copies to the new storage data up to 'elementsAppended'. This variable is only updated when using the `ColumnVector.appendX` API, while `ColumnVector.putX` is more commonly used.
This PR copies the new storage data up to the previously-allocated size in`OffHeapColumnVector`.

## How was this patch tested?

Existing test suites

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

Closes #17811 from kiszk/SPARK-20537.
2017-05-02 13:56:41 +08:00
zero323 90d77e971f [SPARK-20532][SPARKR] Implement grouping and grouping_id
## What changes were proposed in this pull request?

Adds R wrappers for:

- `o.a.s.sql.functions.grouping` as `o.a.s.sql.functions.is_grouping` (to avoid shading `base::grouping`
- `o.a.s.sql.functions.grouping_id`

## How was this patch tested?

Existing unit tests, additional unit tests. `check-cran.sh`.

Author: zero323 <zero323@users.noreply.github.com>

Closes #17807 from zero323/SPARK-20532.
2017-05-01 21:39:17 -07:00
Felix Cheung d20a976e89 [SPARK-20192][SPARKR][DOC] SparkR migration guide to 2.2.0
## What changes were proposed in this pull request?

Updating R Programming Guide

## How was this patch tested?

manually

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #17816 from felixcheung/r22relnote.
2017-05-01 21:03:48 -07:00
Sameer Agarwal 943a684b98 [SPARK-20548] Disable ReplSuite.newProductSeqEncoder with REPL defined class
## What changes were proposed in this pull request?

`newProductSeqEncoder with REPL defined class` in `ReplSuite` has been failing in-deterministically : https://spark-tests.appspot.com/failed-tests over the last few days. Disabling the test until a fix is in place.

https://spark.test.databricks.com/job/spark-master-test-sbt-hadoop-2.7/176/testReport/junit/org.apache.spark.repl/ReplSuite/newProductSeqEncoder_with_REPL_defined_class/history/

## How was this patch tested?

N/A

Author: Sameer Agarwal <sameerag@cs.berkeley.edu>

Closes #17823 from sameeragarwal/disable-test.
2017-05-01 17:42:53 -07:00
ptkool 259860d23d [SPARK-20463] Add support for IS [NOT] DISTINCT FROM.
## What changes were proposed in this pull request?

Add support for the SQL standard distinct predicate to SPARK SQL.

```
<expression> IS [NOT] DISTINCT FROM <expression>
```

## How was this patch tested?

Tested using unit tests, integration tests, manual tests.

Author: ptkool <michael.styles@shopify.com>

Closes #17764 from ptkool/is_not_distinct_from.
2017-05-01 17:05:35 -07:00
Sean Owen af726cd611 [SPARK-20459][SQL] JdbcUtils throws IllegalStateException: Cause already initialized after getting SQLException
## What changes were proposed in this pull request?

Avoid failing to initCause on JDBC exception with cause initialized to null

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #17800 from srowen/SPARK-20459.
2017-05-01 17:01:05 -07:00
Ryan Blue 2b2dd08e97 [SPARK-20540][CORE] Fix unstable executor requests.
There are two problems fixed in this commit. First, the
ExecutorAllocationManager sets a timeout to avoid requesting executors
too often. However, the timeout is always updated based on its value and
a timeout, not the current time. If the call is delayed by locking for
more than the ongoing scheduler timeout, the manager will request more
executors on every run. This seems to be the main cause of SPARK-20540.

The second problem is that the total number of requested executors is
not tracked by the CoarseGrainedSchedulerBackend. Instead, it calculates
the value based on the current status of 3 variables: the number of
known executors, the number of executors that have been killed, and the
number of pending executors. But, the number of pending executors is
never less than 0, even though there may be more known than requested.
When executors are killed and not replaced, this can cause the request
sent to YARN to be incorrect because there were too many executors due
to the scheduler's state being slightly out of date. This is fixed by tracking
the currently requested size explicitly.

## How was this patch tested?

Existing tests.

Author: Ryan Blue <blue@apache.org>

Closes #17813 from rdblue/SPARK-20540-fix-dynamic-allocation.
2017-05-01 14:48:02 -07:00
Kunal Khamar 6fc6cf88d8 [SPARK-20464][SS] Add a job group and description for streaming queries and fix cancellation of running jobs using the job group
## What changes were proposed in this pull request?

Job group: adding a job group is required to properly cancel running jobs related to a query.
Description: the new description makes it easier to group the batches of a query by sorting by name in the Spark Jobs UI.

## How was this patch tested?

- Unit tests
- UI screenshot

  - Order by job id:
![screen shot 2017-04-27 at 5 10 09 pm](https://cloud.githubusercontent.com/assets/7865120/25509468/15452274-2b6e-11e7-87ba-d929816688cf.png)

  - Order by description:
![screen shot 2017-04-27 at 5 10 22 pm](https://cloud.githubusercontent.com/assets/7865120/25509474/1c298512-2b6e-11e7-99b8-fef1ef7665c1.png)

  - Order by job id (no query name):
![screen shot 2017-04-27 at 5 21 33 pm](https://cloud.githubusercontent.com/assets/7865120/25509482/28c96dc8-2b6e-11e7-8df0-9d3cdbb05e36.png)

  - Order by description (no query name):
![screen shot 2017-04-27 at 5 21 44 pm](https://cloud.githubusercontent.com/assets/7865120/25509489/37674742-2b6e-11e7-9357-b5c38ec16ac4.png)

Author: Kunal Khamar <kkhamar@outlook.com>

Closes #17765 from kunalkhamar/sc-6696.
2017-05-01 11:37:30 -07:00
jerryshao ab30590f44 [SPARK-20517][UI] Fix broken history UI download link
The download link in history server UI is concatenated with:

```
 <td><a href="{{uiroot}}/api/v1/applications/{{id}}/{{num}}/logs" class="btn btn-info btn-mini">Download</a></td>
```

Here `num` field represents number of attempts, this is not equal to REST APIs. In the REST API, if attempt id is not existed the URL should be `api/v1/applications/<id>/logs`, otherwise the URL should be `api/v1/applications/<id>/<attemptId>/logs`. Using `<num>` to represent `<attemptId>` will lead to the issue of "no such app".

Manual verification.

CC ajbozarth can you please review this change, since you add this feature before? Thanks!

Author: jerryshao <sshao@hortonworks.com>

Closes #17795 from jerryshao/SPARK-20517.
2017-05-01 10:26:08 -07:00
Herman van Hovell 6b44c4d63a [SPARK-20534][SQL] Make outer generate exec return empty rows
## What changes were proposed in this pull request?
Generate exec does not produce `null` values if the generator for the input row is empty and the generate operates in outer mode without join. This is caused by the fact that the `join=false` code path is different from the `join=true` code path, and that the `join=false` code path did deal with outer properly. This PR addresses this issue.

## How was this patch tested?
Updated `outer*` tests in `GeneratorFunctionSuite`.

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

Closes #17810 from hvanhovell/SPARK-20534.
2017-05-01 09:46:35 -07:00
zero323 f0169a1c6a [SPARK-20290][MINOR][PYTHON][SQL] Add PySpark wrapper for eqNullSafe
## What changes were proposed in this pull request?

Adds Python bindings for `Column.eqNullSafe`

## How was this patch tested?

Manual tests, existing unit tests, doc build.

Author: zero323 <zero323@users.noreply.github.com>

Closes #17605 from zero323/SPARK-20290.
2017-05-01 09:43:32 -07:00
Felix Cheung a355b667a3 [SPARK-20541][SPARKR][SS] support awaitTermination without timeout
## What changes were proposed in this pull request?

Add without param for timeout - will need this to submit a job that runs until stopped
Need this for 2.2

## How was this patch tested?

manually, unit test

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #17815 from felixcheung/rssawaitinfinite.
2017-04-30 23:23:49 -07:00
zero323 80e9cf1b59 [SPARK-20490][SPARKR] Add R wrappers for eqNullSafe and ! / not
## What changes were proposed in this pull request?

- Add null-safe equality operator `%<=>%` (sames as `o.a.s.sql.Column.eqNullSafe`, `o.a.s.sql.Column.<=>`)
- Add boolean negation operator `!` and function `not `.

## How was this patch tested?

Existing unit tests, additional unit tests, `check-cran.sh`.

Author: zero323 <zero323@users.noreply.github.com>

Closes #17783 from zero323/SPARK-20490.
2017-04-30 22:07:12 -07:00
Srinivasa Reddy Vundela 6613046c8c [MINOR][DOCS][PYTHON] Adding missing boolean type for replacement value in fillna
## What changes were proposed in this pull request?

Currently pyspark Dataframe.fillna API supports boolean type when we pass dict, but it is missing in documentation.

## How was this patch tested?
>>> spark.createDataFrame([Row(a=True),Row(a=None)]).fillna({"a" : True}).show()
+----+
|   a|
+----+
|true|
|true|
+----+

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

Author: Srinivasa Reddy Vundela <vsr@cloudera.com>

Closes #17688 from vundela/fillna_doc_fix.
2017-04-30 21:42:05 -07:00
zero323 ae3df4e98f [SPARK-20535][SPARKR] R wrappers for explode_outer and posexplode_outer
## What changes were proposed in this pull request?

Ad R wrappers for

- `o.a.s.sql.functions.explode_outer`
- `o.a.s.sql.functions.posexplode_outer`

## How was this patch tested?

Additional unit tests, manual testing.

Author: zero323 <zero323@users.noreply.github.com>

Closes #17809 from zero323/SPARK-20535.
2017-04-30 12:33:03 -07:00
hyukjinkwon 1ee494d086 [SPARK-20492][SQL] Do not print empty parentheses for invalid primitive types in parser
## What changes were proposed in this pull request?

Currently, when the type string is invalid, it looks printing empty parentheses. This PR proposes a small improvement in an error message by removing it in the parse as below:

```scala
spark.range(1).select($"col".cast("aa"))
```

**Before**

```
org.apache.spark.sql.catalyst.parser.ParseException:
DataType aa() is not supported.(line 1, pos 0)

== SQL ==
aa
^^^
```

**After**

```
org.apache.spark.sql.catalyst.parser.ParseException:
DataType aa is not supported.(line 1, pos 0)

== SQL ==
aa
^^^
```

## How was this patch tested?

Unit tests in `DataTypeParserSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17784 from HyukjinKwon/SPARK-20492.
2017-04-30 08:24:10 -07:00
郭小龙 10207633 4d99b95ad0 [SPARK-20521][DOC][CORE] The default of 'spark.worker.cleanup.appDataTtl' should be 604800 in spark-standalone.md
## What changes were proposed in this pull request?

Currently, our project needs to be set to clean up the worker directory cleanup cycle is three days.
When I follow http://spark.apache.org/docs/latest/spark-standalone.html, configure the 'spark.worker.cleanup.appDataTtl' parameter, I configured to 3 * 24 * 3600.
When I start the spark service, the startup fails, and the worker log displays the error log as follows:

2017-04-28 15:02:03,306 INFO Utils: Successfully started service 'sparkWorker' on port 48728.
Exception in thread "main" java.lang.NumberFormatException: For input string: "3 * 24 * 3600"
	at java.lang.NumberFormatException.forInputString(NumberFormatException.java:65)
	at java.lang.Long.parseLong(Long.java:430)
	at java.lang.Long.parseLong(Long.java:483)
	at scala.collection.immutable.StringLike$class.toLong(StringLike.scala:276)
	at scala.collection.immutable.StringOps.toLong(StringOps.scala:29)
	at org.apache.spark.SparkConf$$anonfun$getLong$2.apply(SparkConf.scala:380)
	at org.apache.spark.SparkConf$$anonfun$getLong$2.apply(SparkConf.scala:380)
	at scala.Option.map(Option.scala:146)
	at org.apache.spark.SparkConf.getLong(SparkConf.scala:380)
	at org.apache.spark.deploy.worker.Worker.<init>(Worker.scala:100)
	at org.apache.spark.deploy.worker.Worker$.startRpcEnvAndEndpoint(Worker.scala:730)
	at org.apache.spark.deploy.worker.Worker$.main(Worker.scala:709)
	at org.apache.spark.deploy.worker.Worker.main(Worker.scala)

**Because we put 7 * 24 * 3600 as a string, forced to convert to the dragon type,  will lead to problems in the program.**

**So I think the default value of the current configuration should be a specific long value, rather than 7 * 24 * 3600,should be 604800. Because it would mislead users for similar configurations, resulting in spark start failure.**

## How was this patch tested?
manual tests

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

Author: 郭小龙 10207633 <guo.xiaolong1@zte.com.cn>
Author: guoxiaolong <guo.xiaolong1@zte.com.cn>
Author: guoxiaolongzte <guo.xiaolong1@zte.com.cn>

Closes #17798 from guoxiaolongzte/SPARK-20521.
2017-04-30 09:06:25 +01:00
hyukjinkwon d228cd0b02 [SPARK-20442][PYTHON][DOCS] Fill up documentations for functions in Column API in PySpark
## What changes were proposed in this pull request?

This PR proposes to fill up the documentation with examples for `bitwiseOR`, `bitwiseAND`, `bitwiseXOR`. `contains`, `asc` and `desc` in `Column` API.

Also, this PR fixes minor typos in the documentation and matches some of the contents between Scala doc and Python doc.

Lastly, this PR suggests to use `spark` rather than `sc` in doc tests in `Column` for Python documentation.

## How was this patch tested?

Doc tests were added and manually tested with the commands below:

`./python/run-tests.py --module pyspark-sql`
`./python/run-tests.py --module pyspark-sql --python-executable python3`
`./dev/lint-python`

Output was checked via `make html` under `./python/docs`. The snapshots will be left on the codes with comments.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17737 from HyukjinKwon/SPARK-20442.
2017-04-29 13:46:40 -07:00
hyukjinkwon 70f1bcd7bc [SPARK-20493][R] De-duplicate parse logics for DDL-like type strings in R
## What changes were proposed in this pull request?

It seems we are using `SQLUtils.getSQLDataType` for type string in structField. It looks we can replace this with `CatalystSqlParser.parseDataType`.

They look similar DDL-like type definitions as below:

```scala
scala> Seq(Tuple1(Tuple1("a"))).toDF.show()
```
```
+---+
| _1|
+---+
|[a]|
+---+
```

```scala
scala> Seq(Tuple1(Tuple1("a"))).toDF.select($"_1".cast("struct<_1:string>")).show()
```
```
+---+
| _1|
+---+
|[a]|
+---+
```

Such type strings looks identical when R’s one as below:

```R
> write.df(sql("SELECT named_struct('_1', 'a') as struct"), "/tmp/aa", "parquet")
> collect(read.df("/tmp/aa", "parquet", structType(structField("struct", "struct<_1:string>"))))
  struct
1      a
```

R’s one is stricter because we are checking the types via regular expressions in R side ahead.

Actual logics there look a bit different but as we check it ahead in R side, it looks replacing it would not introduce (I think) no behaviour changes. To make this sure, the tests dedicated for it were added in SPARK-20105. (It looks `structField` is the only place that calls this method).

## How was this patch tested?

Existing tests - https://github.com/apache/spark/blob/master/R/pkg/inst/tests/testthat/test_sparkSQL.R#L143-L194 should cover this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17785 from HyukjinKwon/SPARK-20493.
2017-04-29 11:02:17 -07:00
wangmiao1981 ee694cdff6 [SPARK-20533][SPARKR] SparkR Wrappers Model should be private and value should be lazy
## What changes were proposed in this pull request?

MultilayerPerceptronClassifierWrapper model should be private.
LogisticRegressionWrapper.scala rFeatures and rCoefficients should be lazy.

## How was this patch tested?

Unit tests.

Author: wangmiao1981 <wm624@hotmail.com>

Closes #17808 from wangmiao1981/lazy.
2017-04-29 10:58:48 -07:00
Yuhao Yang add9d1bba5 [SPARK-19791][ML] Add doc and example for fpgrowth
## What changes were proposed in this pull request?

Add a new section for fpm
Add Example for FPGrowth in scala and Java

updated: Rewrite transform to be more compact.

## How was this patch tested?

local doc generation.

Author: Yuhao Yang <yuhao.yang@intel.com>

Closes #17130 from hhbyyh/fpmdoc.
2017-04-29 10:51:45 -07:00
wangmiao1981 b28c3bc202 [SPARK-20477][SPARKR][DOC] Document R bisecting k-means in R programming guide
## What changes were proposed in this pull request?

Add hyper link in the SparkR programming guide.

## How was this patch tested?

Build doc and manually check the doc link.

Author: wangmiao1981 <wm624@hotmail.com>

Closes #17805 from wangmiao1981/doc.
2017-04-29 10:31:01 -07:00
Tejas Patil 814a61a867 [SPARK-20487][SQL] Display serde for HiveTableScan node in explained plan
## What changes were proposed in this pull request?

This was a suggestion by rxin at https://github.com/apache/spark/pull/17780#issuecomment-298073408

## How was this patch tested?

- modified existing unit test
- manual testing:

```
scala> hc.sql(" SELECT * FROM tejasp_bucketed_partitioned_1  where name = ''  ").explain(true)
== Parsed Logical Plan ==
'Project [*]
+- 'Filter ('name = )
   +- 'UnresolvedRelation `tejasp_bucketed_partitioned_1`

== Analyzed Logical Plan ==
user_id: bigint, name: string, ds: string
Project [user_id#24L, name#25, ds#26]
+- Filter (name#25 = )
   +- SubqueryAlias tejasp_bucketed_partitioned_1
      +- CatalogRelation `default`.`tejasp_bucketed_partitioned_1`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [user_id#24L, name#25], [ds#26]

== Optimized Logical Plan ==
Filter (isnotnull(name#25) && (name#25 = ))
+- CatalogRelation `default`.`tejasp_bucketed_partitioned_1`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [user_id#24L, name#25], [ds#26]

== Physical Plan ==
*Filter (isnotnull(name#25) && (name#25 = ))
+- HiveTableScan [user_id#24L, name#25, ds#26], CatalogRelation `default`.`tejasp_bucketed_partitioned_1`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [user_id#24L, name#25], [ds#26]
```

Author: Tejas Patil <tejasp@fb.com>

Closes #17806 from tejasapatil/add_serde.
2017-04-28 23:12:26 -07:00
Aaditya Ramesh 77bcd77ed5 [SPARK-19525][CORE] Add RDD checkpoint compression support
## What changes were proposed in this pull request?

This PR adds RDD checkpoint compression support and add a new config `spark.checkpoint.compress` to enable/disable it. Credit goes to aramesh117

Closes #17024

## How was this patch tested?

The new unit test.

Author: Shixiong Zhu <shixiong@databricks.com>
Author: Aaditya Ramesh <aramesh@conviva.com>

Closes #17789 from zsxwing/pr17024.
2017-04-28 15:28:56 -07:00
caoxuewen ebff519c5e [SPARK-20471] Remove AggregateBenchmark testsuite warning: Two level hashmap is disabled but vectorized hashmap is enabled
What changes were proposed in this pull request?

remove  AggregateBenchmark testsuite warning:
such as '14:26:33.220 WARN org.apache.spark.sql.execution.aggregate.HashAggregateExec: Two level hashmap is disabled but vectorized hashmap is enabled.'

How was this patch tested?
unit tests: AggregateBenchmark
Modify the 'ignore function for 'test funtion

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

Closes #17771 from heary-cao/AggregateBenchmark.
2017-04-28 14:47:17 -07:00
Mark Grover 5d71f3db83 [SPARK-20514][CORE] Upgrade Jetty to 9.3.11.v20160721
Upgrade Jetty so it can work with Hadoop 3 (alpha 2 release, in particular).
Without this change, because of incompatibily between Jetty versions,
Spark fails to compile when built against Hadoop 3

## How was this patch tested?
Unit tests being run.

Author: Mark Grover <mark@apache.org>

Closes #17790 from markgrover/spark-20514.
2017-04-28 14:06:57 -07:00
Bill Chambers 733b81b835 [SPARK-20496][SS] Bug in KafkaWriter Looks at Unanalyzed Plans
## What changes were proposed in this pull request?

We didn't enforce analyzed plans in Spark 2.1 when writing out to Kafka.

## How was this patch tested?

New unit test.

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

Author: Bill Chambers <bill@databricks.com>

Closes #17804 from anabranch/SPARK-20496-2.
2017-04-28 10:18:31 -07:00
hyukjinkwon 8c911adac5 [SPARK-20465][CORE] Throws a proper exception when any temp directory could not be got
## What changes were proposed in this pull request?

This PR proposes to throw an exception with better message rather than `ArrayIndexOutOfBoundsException` when temp directories could not be created.

Running the commands below:

```bash
./bin/spark-shell --conf spark.local.dir=/NONEXISTENT_DIR_ONE,/NONEXISTENT_DIR_TWO
```

produces ...

**Before**

```
Exception in thread "main" java.lang.ExceptionInInitializerError
        ...
Caused by: java.lang.ArrayIndexOutOfBoundsException: 0
        ...
```

**After**

```
Exception in thread "main" java.lang.ExceptionInInitializerError
        ...
Caused by: java.io.IOException: Failed to get a temp directory under [/NONEXISTENT_DIR_ONE,/NONEXISTENT_DIR_TWO].
        ...
```

## How was this patch tested?

Unit tests in `LocalDirsSuite.scala`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17768 from HyukjinKwon/throws-temp-dir-exception.
2017-04-28 08:49:35 +01:00