Commit graph

18433 commits

Author SHA1 Message Date
gatorsmile 422a45cf04 [SPARK-18766][SQL] Push Down Filter Through BatchEvalPython (Python UDF)
### What changes were proposed in this pull request?
Currently, when users use Python UDF in Filter, BatchEvalPython is always generated below FilterExec. However, not all the predicates need to be evaluated after Python UDF execution. Thus, this PR is to push down the determinisitc predicates through `BatchEvalPython`.
```Python
>>> df = spark.createDataFrame([(1, "1"), (2, "2"), (1, "2"), (1, "2")], ["key", "value"])
>>> from pyspark.sql.functions import udf, col
>>> from pyspark.sql.types import BooleanType
>>> my_filter = udf(lambda a: a < 2, BooleanType())
>>> sel = df.select(col("key"), col("value")).filter((my_filter(col("key"))) & (df.value < "2"))
>>> sel.explain(True)
```
Before the fix, the plan looks like
```
== Optimized Logical Plan ==
Filter ((isnotnull(value#1) && <lambda>(key#0L)) && (value#1 < 2))
+- LogicalRDD [key#0L, value#1]

== Physical Plan ==
*Project [key#0L, value#1]
+- *Filter ((isnotnull(value#1) && pythonUDF0#9) && (value#1 < 2))
   +- BatchEvalPython [<lambda>(key#0L)], [key#0L, value#1, pythonUDF0#9]
      +- Scan ExistingRDD[key#0L,value#1]
```

After the fix, the plan looks like
```
== Optimized Logical Plan ==
Filter ((isnotnull(value#1) && <lambda>(key#0L)) && (value#1 < 2))
+- LogicalRDD [key#0L, value#1]

== Physical Plan ==
*Project [key#0L, value#1]
+- *Filter pythonUDF0#9: boolean
   +- BatchEvalPython [<lambda>(key#0L)], [key#0L, value#1, pythonUDF0#9]
      +- *Filter (isnotnull(value#1) && (value#1 < 2))
         +- Scan ExistingRDD[key#0L,value#1]
```

### How was this patch tested?
Added both unit test cases for `BatchEvalPythonExec` and also add an end-to-end test case in Python test suite.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16193 from gatorsmile/pythonUDFPredicatePushDown.
2016-12-10 08:47:45 -08:00
WangTaoTheTonic 3a3e65adaf
[SPARK-18606][HISTORYSERVER] remove useless elements while searching
## What changes were proposed in this pull request?

When we search applications in HistoryServer, it will include all contents between <td> tag, which including useless elemtns like "<span title...", "a href" and making results confused.
We should remove those to make it clear.

## How was this patch tested?

manual tests.

Before:
![before](https://cloud.githubusercontent.com/assets/5276001/20662840/28bcc874-b590-11e6-9115-12fb64e49898.jpg)

After:
![after](https://cloud.githubusercontent.com/assets/5276001/20662844/2f717af2-b590-11e6-97dc-a48b08a54247.jpg)

Author: WangTaoTheTonic <wangtao111@huawei.com>

Closes #16031 from WangTaoTheTonic/span.
2016-12-10 16:43:08 +00:00
Dongjoon Hyun f3a3fed76c
[MINOR][DOCS] Remove Apache Spark Wiki address
## What changes were proposed in this pull request?

According to the notice of the following Wiki front page, we can remove the obsolete wiki pointer safely in `README.md` and `docs/index.md`, too. These two lines are the last occurrence of that links.

```
All current wiki content has been merged into pages at http://spark.apache.org as of November 2016.
Each page links to the new location of its information on the Spark web site.
Obsolete wiki content is still hosted here, but carries a notice that it is no longer current.
```

## How was this patch tested?

Manual.

- `README.md`: https://github.com/dongjoon-hyun/spark/tree/remove_wiki_from_readme
- `docs/index.md`:
```
cd docs
SKIP_API=1 jekyll build
```
![screen shot 2016-12-09 at 2 53 29 pm](https://cloud.githubusercontent.com/assets/9700541/21067323/517252e2-be1f-11e6-85b1-2a4471131c5d.png)

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #16239 from dongjoon-hyun/remove_wiki_from_readme.
2016-12-10 16:40:10 +00:00
Huaxin Gao c5172568b5 [SPARK-17460][SQL] Make sure sizeInBytes in Statistics will not overflow
## What changes were proposed in this pull request?

1. In SparkStrategies.canBroadcast, I will add the check   plan.statistics.sizeInBytes >= 0
2. In LocalRelations.statistics, when calculate the statistics, I will change the size to BigInt so it won't overflow.

## How was this patch tested?

I will add a test case to make sure the statistics.sizeInBytes won't overflow.

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

Closes #16175 from huaxingao/spark-17460.
2016-12-10 22:41:40 +08:00
Burak Yavuz 63c9159870 [SPARK-18811] StreamSource resolution should happen in stream execution thread
## What changes were proposed in this pull request?

When you start a stream, if we are trying to resolve the source of the stream, for example if we need to resolve partition columns, this could take a long time. This long execution time should not block the main thread where `query.start()` was called on. It should happen in the stream execution thread possibly before starting any triggers.

## How was this patch tested?

Unit test added. Made sure test fails with no code changes.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #16238 from brkyvz/SPARK-18811.
2016-12-09 22:49:51 -08:00
Felix Cheung 3e11d5bfef [SPARK-18807][SPARKR] Should suppress output print for calls to JVM methods with void return values
## What changes were proposed in this pull request?

Several SparkR API calling into JVM methods that have void return values are getting printed out, especially when running in a REPL or IDE.
example:
```
> setLogLevel("WARN")
NULL
```
We should fix this to make the result more clear.

Also found a small change to return value of dropTempView in 2.1 - adding doc and test for it.

## How was this patch tested?

manually - I didn't find a expect_*() method in testthat for this

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #16237 from felixcheung/rinvis.
2016-12-09 19:06:05 -08:00
Xiangrui Meng d2493a203e [SPARK-18812][MLLIB] explain "Spark ML"
## What changes were proposed in this pull request?

There has been some confusion around "Spark ML" vs. "MLlib". This PR adds some FAQ-like entries to the MLlib user guide to explain "Spark ML" and reduce the confusion.

I check the [Spark FAQ page](http://spark.apache.org/faq.html), which seems too high-level for the content here. So I added it to the MLlib user guide instead.

cc: mateiz

Author: Xiangrui Meng <meng@databricks.com>

Closes #16241 from mengxr/SPARK-18812.
2016-12-09 17:34:52 -08:00
Davies Liu cf33a86285 [SPARK-4105] retry the fetch or stage if shuffle block is corrupt
## What changes were proposed in this pull request?

There is an outstanding issue that existed for a long time: Sometimes the shuffle blocks are corrupt and can't be decompressed. We recently hit this in three different workloads, sometimes we can reproduce it by every try, sometimes can't. I also found that when the corruption happened, the beginning and end of the blocks are correct, the corruption happen in the middle. There was one case that the string of block id is corrupt by one character. It seems that it's very likely the corruption is introduced by some weird machine/hardware, also the checksum (16 bits) in TCP is not strong enough to identify all the corruption.

Unfortunately, Spark does not have checksum for shuffle blocks or broadcast, the job will fail if any corruption happen in the shuffle block from disk, or broadcast blocks during network. This PR try to detect the corruption after fetching shuffle blocks by decompressing them, because most of the compression already have checksum in them. It will retry the block, or failed with FetchFailure, so the previous stage could be retried on different (still random) machines.

Checksum for broadcast will be added by another PR.

## How was this patch tested?

Added unit tests

Author: Davies Liu <davies@databricks.com>

Closes #15923 from davies/detect_corrupt.
2016-12-09 15:44:22 -08:00
Kazuaki Ishizaki d60ab5fd9b [SPARK-18745][SQL] Fix signed integer overflow due to toInt cast
## What changes were proposed in this pull request?

This PR avoids that a result of a cast `toInt` is negative due to signed integer overflow (e.g. 0x0000_0000_1???????L.toInt < 0 ). This PR performs casts after we can ensure the value is within range of signed integer (the result of `max(array.length, ???)` is always integer).

## How was this patch tested?

Manually executed query68 of TPC-DS with 100TB

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

Closes #16235 from kiszk/SPARK-18745.
2016-12-09 23:13:36 +01:00
Takeshi YAMAMURO b08b500456
[SPARK-18620][STREAMING][KINESIS] Flatten input rates in timeline for streaming + kinesis
## What changes were proposed in this pull request?
This pr is to make input rates in timeline more flat for spark streaming + kinesis.
Since kinesis workers fetch records and push them into block generators in bulk, timeline in web UI has many spikes when `maxRates` applied (See a Figure.1 below). This fix splits fetched input records into multiple `adRecords` calls.

Figure.1 Apply `maxRates=500` in vanilla Spark
<img width="1084" alt="apply_limit in_vanilla_spark" src="https://cloud.githubusercontent.com/assets/692303/20823861/4602f300-b89b-11e6-95f3-164a37061305.png">

Figure.2 Apply `maxRates=500` in Spark with my patch
<img width="1056" alt="apply_limit in_spark_with_my_patch" src="https://cloud.githubusercontent.com/assets/692303/20823882/6c46352c-b89b-11e6-81ab-afd8abfe0cfe.png">

## How was this patch tested?
Add tests to check to split input records into multiple `addRecords` calls.

Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes #16114 from maropu/SPARK-18620.
2016-12-10 05:32:04 +08:00
Shivaram Venkataraman be5fc6ef72 [MINOR][SPARKR] Fix SparkR regex in copy command
Fix SparkR package copy regex. The existing code leads to
```
Copying release tarballs to /home/****/public_html/spark-nightly/spark-branch-2.1-bin/spark-2.1.1-SNAPSHOT-2016_12_08_22_38-e8f351f-bin
mput: SparkR-*: no files found
```

Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu>

Closes #16231 from shivaram/typo-sparkr-build.
2016-12-09 10:12:56 -08:00
Xiangrui Meng fd48d80a61 [SPARK-17822][R] Make JVMObjectTracker a member variable of RBackend
## What changes were proposed in this pull request?

* This PR changes `JVMObjectTracker` from `object` to `class` and let its instance associated with each RBackend. So we can manage the lifecycle of JVM objects when there are multiple `RBackend` sessions. `RBackend.close` will clear the object tracker explicitly.
* I assume that `SQLUtils` and `RRunner` do not need to track JVM instances, which could be wrong.
* Small refactor of `SerDe.sqlSerDe` to increase readability.

## How was this patch tested?

* Added unit tests for `JVMObjectTracker`.
* Wait for Jenkins to run full tests.

Author: Xiangrui Meng <meng@databricks.com>

Closes #16154 from mengxr/SPARK-17822.
2016-12-09 07:51:46 -08:00
Jacek Laskowski b162cc0c28
[MINOR][CORE][SQL][DOCS] Typo fixes
## What changes were proposed in this pull request?

Typo fixes

## How was this patch tested?

Local build. Awaiting the official build.

Author: Jacek Laskowski <jacek@japila.pl>

Closes #16144 from jaceklaskowski/typo-fixes.
2016-12-09 18:45:57 +08:00
Zhan Zhang 67587d961d [SPARK-18637][SQL] Stateful UDF should be considered as nondeterministic
## What changes were proposed in this pull request?

Make stateful udf as nondeterministic

## How was this patch tested?
Add new test cases with both Stateful and Stateless UDF.
Without the patch, the test cases will throw exception:

1 did not equal 10
ScalaTestFailureLocation: org.apache.spark.sql.hive.execution.HiveUDFSuite$$anonfun$21 at (HiveUDFSuite.scala:501)
org.scalatest.exceptions.TestFailedException: 1 did not equal 10
        at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500)
        at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555)
        ...

Author: Zhan Zhang <zhanzhang@fb.com>

Closes #16068 from zhzhan/state.
2016-12-09 16:35:06 +08:00
Felix Cheung c074c96dc5 Copy pyspark and SparkR packages to latest release dir too
## What changes were proposed in this pull request?

Copy pyspark and SparkR packages to latest release dir, as per comment [here](https://github.com/apache/spark/pull/16226#discussion_r91664822)

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #16227 from felixcheung/pyrftp.
2016-12-08 22:52:34 -08:00
Shivaram Venkataraman 934035ae7c Copy the SparkR source package with LFTP
This PR adds a line in release-build.sh to copy the SparkR source archive using LFTP

Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu>

Closes #16226 from shivaram/fix-sparkr-copy-build.
2016-12-08 22:21:24 -08:00
Weiqing Yang 9338aa4f89
[SPARK-18697][BUILD] Upgrade sbt plugins
## What changes were proposed in this pull request?

This PR is to upgrade sbt plugins. The following sbt plugins will be upgraded:
```
sbteclipse-plugin: 4.0.0 -> 5.0.1
sbt-mima-plugin: 0.1.11 -> 0.1.12
org.ow2.asm/asm: 5.0.3 -> 5.1
org.ow2.asm/asm-commons: 5.0.3 -> 5.1
```
## How was this patch tested?
Pass the Jenkins build.

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #16223 from weiqingy/SPARK_18697.
2016-12-09 14:13:01 +08:00
wm624@hotmail.com 86a96034cc [SPARK-18349][SPARKR] Update R API documentation on ml model summary
## What changes were proposed in this pull request?
In this PR, the document of `summary` method is improved in the format:

returns summary information of the fitted model, which is a list. The list includes .......

Since `summary` in R is mainly about the model, which is not the same as `summary` object on scala side, if there is one, the scala API doc is not pointed here.

In current document, some `return` have `.` and some don't have. `.` is added to missed ones.

Since spark.logit `summary` has a big refactoring, this PR doesn't include this one. It will be changed when the `spark.logit` PR is merged.

## How was this patch tested?

Manual build.

Author: wm624@hotmail.com <wm624@hotmail.com>

Closes #16150 from wangmiao1981/audit2.
2016-12-08 22:08:19 -08:00
Shivaram Venkataraman 4ac8b20bf2 [SPARKR][PYSPARK] Fix R source package name to match Spark version. Remove pip tar.gz from distribution
## What changes were proposed in this pull request?

Fixes name of R source package so that the `cp` in release-build.sh works correctly.

Issue discussed in https://github.com/apache/spark/pull/16014#issuecomment-265867125

Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu>

Closes #16221 from shivaram/fix-sparkr-release-build-name.
2016-12-08 18:26:54 -08:00
Tathagata Das 458fa3325e [SPARK-18776][SS] Make Offset for FileStreamSource corrected formatted in json
## What changes were proposed in this pull request?

- Changed FileStreamSource to use new FileStreamSourceOffset rather than LongOffset. The field is named as `logOffset` to make it more clear that this is a offset in the file stream log.
- Fixed bug in FileStreamSourceLog, the field endId in the FileStreamSourceLog.get(startId, endId) was not being used at all. No test caught it earlier. Only my updated tests caught it.

Other minor changes
- Dont use batchId in the FileStreamSource, as calling it batch id is extremely miss leading. With multiple sources, it may happen that a new batch has no new data from a file source. So offset of FileStreamSource != batchId after that batch.

## How was this patch tested?

Updated unit test.

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

Closes #16205 from tdas/SPARK-18776.
2016-12-08 17:53:34 -08:00
Shivaram Venkataraman 202fcd21ce [SPARK-18590][SPARKR] Change the R source build to Hadoop 2.6
This PR changes the SparkR source release tarball to be built using the Hadoop 2.6 profile. Previously it was using the without hadoop profile which leads to an error as discussed in https://github.com/apache/spark/pull/16014#issuecomment-265843991

Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu>

Closes #16218 from shivaram/fix-sparkr-release-build.
2016-12-08 13:01:46 -08:00
Reynold Xin 3261e25da3 Close stale PRs.
Closes #16191
Closes #16198
Closes #14561
Closes #14223
Closes #7739
Closes #13026
Closes #16217
2016-12-08 12:56:08 -08:00
Reynold Xin 5f894d23a5 [SPARK-18760][SQL] Consistent format specification for FileFormats
## What changes were proposed in this pull request?
This patch fixes the format specification in explain for file sources (Parquet and Text formats are the only two that are different from the rest):

Before:
```
scala> spark.read.text("test.text").explain()
== Physical Plan ==
*FileScan text [value#15] Batched: false, Format: org.apache.spark.sql.execution.datasources.text.TextFileFormatxyz, Location: InMemoryFileIndex[file:/scratch/rxin/spark/test.text], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<value:string>
```

After:
```
scala> spark.read.text("test.text").explain()
== Physical Plan ==
*FileScan text [value#15] Batched: false, Format: Text, Location: InMemoryFileIndex[file:/scratch/rxin/spark/test.text], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<value:string>
```

Also closes #14680.

## How was this patch tested?
Verified in spark-shell.

Author: Reynold Xin <rxin@databricks.com>

Closes #16187 from rxin/SPARK-18760.
2016-12-08 12:52:05 -08:00
Shixiong Zhu 26432df9cc [SPARK-18751][CORE] Fix deadlock when SparkContext.stop is called in Utils.tryOrStopSparkContext
## What changes were proposed in this pull request?

When `SparkContext.stop` is called in `Utils.tryOrStopSparkContext` (the following three places), it will cause deadlock because the `stop` method needs to wait for the thread running `stop` to exit.

- ContextCleaner.keepCleaning
- LiveListenerBus.listenerThread.run
- TaskSchedulerImpl.start

This PR adds `SparkContext.stopInNewThread` and uses it to eliminate the potential deadlock. I also removed my changes in #15775 since they are not necessary now.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16178 from zsxwing/fix-stop-deadlock.
2016-12-08 11:54:04 -08:00
Felix Cheung c3d3a9d0e8 [SPARK-18590][SPARKR] build R source package when making distribution
## What changes were proposed in this pull request?

This PR has 2 key changes. One, we are building source package (aka bundle package) for SparkR which could be released on CRAN. Two, we should include in the official Spark binary distributions SparkR installed from this source package instead (which would have help/vignettes rds needed for those to work when the SparkR package is loaded in R, whereas earlier approach with devtools does not)

But, because of various differences in how R performs different tasks, this PR is a fair bit more complicated. More details below.

This PR also includes a few minor fixes.

### more details

These are the additional steps in make-distribution; please see [here](https://github.com/apache/spark/blob/master/R/CRAN_RELEASE.md) on what's going to a CRAN release, which is now run during make-distribution.sh.
1. package needs to be installed because the first code block in vignettes is `library(SparkR)` without lib path
2. `R CMD build` will build vignettes (this process runs Spark/SparkR code and captures outputs into pdf documentation)
3. `R CMD check` on the source package will install package and build vignettes again (this time from source packaged) - this is a key step required to release R package on CRAN
 (will skip tests here but tests will need to pass for CRAN release process to success - ideally, during release signoff we should install from the R source package and run tests)
4. `R CMD Install` on the source package (this is the only way to generate doc/vignettes rds files correctly, not in step # 1)
 (the output of this step is what we package into Spark dist and sparkr.zip)

Alternatively,
   R CMD build should already be installing the package in a temp directory though it might just be finding this location and set it to lib.loc parameter; another approach is perhaps we could try calling `R CMD INSTALL --build pkg` instead.
 But in any case, despite installing the package multiple times this is relatively fast.
Building vignettes takes a while though.

## How was this patch tested?

Manually, CI.

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #16014 from felixcheung/rdist.
2016-12-08 11:29:31 -08:00
Andrew Ray 3c68944b22 [SPARK-16589] [PYTHON] Chained cartesian produces incorrect number of records
## What changes were proposed in this pull request?

Fixes a bug in the python implementation of rdd cartesian product related to batching that showed up in repeated cartesian products with seemingly random results. The root cause being multiple iterators pulling from the same stream in the wrong order because of logic that ignored batching.

`CartesianDeserializer` and `PairDeserializer` were changed to implement `_load_stream_without_unbatching` and borrow the one line implementation of `load_stream` from `BatchedSerializer`. The default implementation of `_load_stream_without_unbatching` was changed to give consistent results (always an iterable) so that it could be used without additional checks.

`PairDeserializer` no longer extends `CartesianDeserializer` as it was not really proper. If wanted a new common super class could be added.

Both `CartesianDeserializer` and `PairDeserializer` now only extend `Serializer` (which has no `dump_stream` implementation) since they are only meant for *de*serialization.

## How was this patch tested?

Additional unit tests (sourced from #14248) plus one for testing a cartesian with zip.

Author: Andrew Ray <ray.andrew@gmail.com>

Closes #16121 from aray/fix-cartesian.
2016-12-08 11:08:12 -08:00
Ergin Seyfe ed8869ebbf [SPARK-8617][WEBUI] HistoryServer: Include in-progress files during cleanup
## What changes were proposed in this pull request?
- Removed the`attempt.completed ` filter so cleaner would include the orphan inprogress files.
- Use loading time for inprogress files as lastUpdated. Keep using the modTime for completed files. First one will prevent deletion of inprogress job files. Second one will ensure that lastUpdated time won't change for completed jobs in an event of HistoryServer reboot.

## How was this patch tested?
Added new unittests and via existing tests.

Author: Ergin Seyfe <eseyfe@fb.com>

Closes #16165 from seyfe/clear_old_inprogress_files.
2016-12-08 10:21:09 -08:00
Marcelo Vanzin b44d1b8fcf [SPARK-18662][HOTFIX] Add new resource-managers directories to SparkLauncher.
These directories are added to the classpath of applications when testing or
using SPARK_PREPEND_CLASSES, otherwise updated classes are not seen. Also,
add the mesos directory which was missing.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #16202 from vanzin/SPARK-18662.
2016-12-08 09:48:33 -08:00
Liang-Chi Hsieh 6a5a7254dc [SPARK-18667][PYSPARK][SQL] Change the way to group row in BatchEvalPythonExec so input_file_name function can work with UDF in pyspark
## What changes were proposed in this pull request?

`input_file_name` doesn't return filename when working with UDF in PySpark. An example shows the problem:

    from pyspark.sql.functions import *
    from pyspark.sql.types import *

    def filename(path):
        return path

    sourceFile = udf(filename, StringType())
    spark.read.json("tmp.json").select(sourceFile(input_file_name())).show()

    +---------------------------+
    |filename(input_file_name())|
    +---------------------------+
    |                           |
    +---------------------------+

The cause of this issue is, we group rows in `BatchEvalPythonExec` for batching processing of PythonUDF. Currently we group rows first and then evaluate expressions on the rows. If the data is less than the required number of rows for a group, the iterator will be consumed to the end before the evaluation. However, once the iterator reaches the end, we will unset input filename. So the input_file_name expression can't return correct filename.

This patch fixes the approach to group the batch of rows. We evaluate the expression first and then group evaluated results to batch.

## How was this patch tested?

Added unit test to PySpark.

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

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

Closes #16115 from viirya/fix-py-udf-input-filename.
2016-12-08 23:22:18 +08:00
hyukjinkwon 7f3c778fd0
[SPARK-18718][TESTS] Skip some test failures due to path length limitation and fix tests to pass on Windows
## What changes were proposed in this pull request?

There are some tests failed on Windows due to the wrong format of path and the limitation of path length as below:

This PR proposes both to fix the failed tests by fixing the path for the tests below:

- `InsertSuite`
  ```
  Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.sources.InsertSuite *** ABORTED *** (12 seconds, 547 milliseconds)
      org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-177945ef-9128-42b4-8c07-de31f78bbbd6;
      at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$14.apply(DataSource.scala:382)
      at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$14.apply(DataSource.scala:370)
      at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
  ```

- `PathOptionSuite`
  ```
  - path option also exist for write path *** FAILED *** (1 second, 93 milliseconds)
    "C:[projectsspark	arget	mp]spark-5ab34a58-df8d-..." did not equal "C:[\projects\spark\target\tmp\]spark-5ab34a58-df8d-..." (PathOptionSuite.scala:93)
    org.scalatest.exceptions.TestFailedException:
        at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500)
        at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555)
    ...
  ```

- `UDFSuite`
  ```
  - SPARK-8005 input_file_name *** FAILED *** (2 seconds, 234 milliseconds)
    "file:///C:/projects/spark/target/tmp/spark-e4e5720a-2006-48f9-8b11-797bf59794bf/part-00001-26fb05e4-603d-471d-ae9d-b9549e0c7765.snappy.parquet" did not contain "C:\projects\spark\target\tmp\spark-e4e5720a-2006-48f9-8b11-797bf59794bf" (UDFSuite.scala:67)
    org.scalatest.exceptions.TestFailedException:
      at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500)
      at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555)
    ...
  ```

and to skip the tests belows which are being failed on Windows due to path length limitation.

- `SparkLauncherSuite`
  ```
  Test org.apache.spark.launcher.SparkLauncherSuite.testChildProcLauncher failed: java.lang.AssertionError: expected:<0> but was:<1>, took 0.062 sec
    at org.apache.spark.launcher.SparkLauncherSuite.testChildProcLauncher(SparkLauncherSuite.java:177)
      ...
  ```

  The stderr from the process is `The filename or extension is too long` which is equivalent to the one below.

- `BroadcastJoinSuite`
  ```
  04:09:40.882 ERROR org.apache.spark.deploy.worker.ExecutorRunner: Error running executor
  java.io.IOException: Cannot run program "C:\Progra~1\Java\jdk1.8.0\bin\java" (in directory "C:\projects\spark\work\app-20161205040542-0000\51658"): CreateProcess error=206, The filename or extension is too long
      at java.lang.ProcessBuilder.start(ProcessBuilder.java:1048)
      at org.apache.spark.deploy.worker.ExecutorRunner.org$apache$spark$deploy$worker$ExecutorRunner$$fetchAndRunExecutor(ExecutorRunner.scala:167)
      at org.apache.spark.deploy.worker.ExecutorRunner$$anon$1.run(ExecutorRunner.scala:73)
  Caused by: java.io.IOException: CreateProcess error=206, The filename or extension is too long
      at java.lang.ProcessImpl.create(Native Method)
      at java.lang.ProcessImpl.<init>(ProcessImpl.java:386)
      at java.lang.ProcessImpl.start(ProcessImpl.java:137)
      at java.lang.ProcessBuilder.start(ProcessBuilder.java:1029)
      ... 2 more
  04:09:40.929 ERROR org.apache.spark.deploy.worker.ExecutorRunner: Error running executor

    (appearently infinite same error messages)

  ...
  ```

## How was this patch tested?

Manually tested via AppVeyor.

**Before**

`InsertSuite`: https://ci.appveyor.com/project/spark-test/spark/build/148-InsertSuite-pr
`PathOptionSuite`: https://ci.appveyor.com/project/spark-test/spark/build/139-PathOptionSuite-pr
`UDFSuite`: https://ci.appveyor.com/project/spark-test/spark/build/143-UDFSuite-pr
`SparkLauncherSuite`: https://ci.appveyor.com/project/spark-test/spark/build/141-SparkLauncherSuite-pr
`BroadcastJoinSuite`: https://ci.appveyor.com/project/spark-test/spark/build/145-BroadcastJoinSuite-pr

**After**

`PathOptionSuite`: https://ci.appveyor.com/project/spark-test/spark/build/140-PathOptionSuite-pr
`SparkLauncherSuite`: https://ci.appveyor.com/project/spark-test/spark/build/142-SparkLauncherSuite-pr
`UDFSuite`: https://ci.appveyor.com/project/spark-test/spark/build/144-UDFSuite-pr
`InsertSuite`: https://ci.appveyor.com/project/spark-test/spark/build/147-InsertSuite-pr
`BroadcastJoinSuite`: https://ci.appveyor.com/project/spark-test/spark/build/149-BroadcastJoinSuite-pr

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16147 from HyukjinKwon/fix-tests.
2016-12-08 23:02:05 +08:00
Yanbo Liang 9bf8f3cd4f [SPARK-18325][SPARKR][ML] SparkR ML wrappers example code and user guide
## What changes were proposed in this pull request?
* Add all R examples for ML wrappers which were added during 2.1 release cycle.
* Split the whole ```ml.R``` example file into individual example for each algorithm, which will be convenient for users to rerun them.
* Add corresponding examples to ML user guide.
* Update ML section of SparkR user guide.

Note: MLlib Scala/Java/Python examples will be consistent, however, SparkR examples may different from them, since R users may use the algorithms in a different way, for example, using R ```formula``` to specify ```featuresCol``` and ```labelCol```.

## How was this patch tested?
Run all examples manually.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #16148 from yanboliang/spark-18325.
2016-12-08 06:19:38 -08:00
Shixiong Zhu b47b892e45 [SPARK-18774][CORE][SQL] Ignore non-existing files when ignoreCorruptFiles is enabled
## What changes were proposed in this pull request?

When `ignoreCorruptFiles` is enabled, it's better to also ignore non-existing files.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16203 from zsxwing/ignore-file-not-found.
2016-12-07 22:37:04 -08:00
Reynold Xin 330fda8aa2 Close stale pull requests.
Closes #15689
Closes #14640
Closes #15917
Closes #16188
Closes #16206
2016-12-07 22:29:57 -08:00
Yanbo Liang 97255497d8 [SPARK-18326][SPARKR][ML] Review SparkR ML wrappers API for 2.1
## What changes were proposed in this pull request?
Reviewing SparkR ML wrappers API for 2.1 release, mainly two issues:
* Remove ```probabilityCol``` from the argument list of ```spark.logit``` and ```spark.randomForest```. Since it was used when making prediction and should be an argument of ```predict```, and we will work on this at [SPARK-18618](https://issues.apache.org/jira/browse/SPARK-18618) in the next release cycle.
* Fix ```spark.als``` params to make it consistent with MLlib.

## How was this patch tested?
Existing tests.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #16169 from yanboliang/spark-18326.
2016-12-07 20:23:28 -08:00
sethah 82253617f5 [SPARK-18705][ML][DOC] Update user guide to reflect one pass solver for L1 and elastic-net
## What changes were proposed in this pull request?

WeightedLeastSquares now supports L1 and elastic net penalties and has an additional solver option: QuasiNewton. The docs are updated to reflect this change.

## How was this patch tested?

Docs only. Generated documentation to make sure Latex looks ok.

Author: sethah <seth.hendrickson16@gmail.com>

Closes #16139 from sethah/SPARK-18705.
2016-12-07 19:41:32 -08:00
Tathagata Das 9ab725eabb [SPARK-18758][SS] StreamingQueryListener events from a StreamingQuery should be sent only to the listeners in the same session as the query
## What changes were proposed in this pull request?

Listeners added with `sparkSession.streams.addListener(l)` are added to a SparkSession. So events only from queries in the same session as a listener should be posted to the listener. Currently, all the events gets rerouted through the Spark's main listener bus, that is,
- StreamingQuery posts event to StreamingQueryListenerBus. Only the queries associated with the same session as the bus posts events to it.
- StreamingQueryListenerBus posts event to Spark's main LiveListenerBus as a SparkEvent.
- StreamingQueryListenerBus also subscribes to LiveListenerBus events thus getting back the posted event in a different thread.
- The received is posted to the registered listeners.

The problem is that *all StreamingQueryListenerBuses in all sessions* gets the events and posts them to their listeners. This is wrong.

In this PR, I solve it by making StreamingQueryListenerBus track active queries (by their runIds) when a query posts the QueryStarted event to the bus. This allows the rerouted events to be filtered using the tracked queries.

Note that this list needs to be maintained separately
from the `StreamingQueryManager.activeQueries` because a terminated query is cleared from
`StreamingQueryManager.activeQueries` as soon as it is stopped, but the this ListenerBus must
clear a query only after the termination event of that query has been posted lazily, much after the query has been terminated.

Credit goes to zsxwing for coming up with the initial idea.

## How was this patch tested?
Updated test harness code to use the correct session, and added new unit test.

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

Closes #16186 from tdas/SPARK-18758.
2016-12-07 19:23:27 -08:00
wm624@hotmail.com aad11209eb [SPARK-18633][ML][EXAMPLE] Add multiclass logistic regression summary python example and document
## What changes were proposed in this pull request?
Logistic Regression summary is added in Python API. We need to add example and document for summary.

The newly added example is consistent with Scala and Java examples.

## How was this patch tested?

Manually tests: Run the example with spark-submit; copy & paste code into pyspark; build document and check the document.

Author: wm624@hotmail.com <wm624@hotmail.com>

Closes #16064 from wangmiao1981/py.
2016-12-07 18:12:49 -08:00
Nathan Howell bec0a9217b [SPARK-18654][SQL] Remove unreachable patterns in makeRootConverter
## What changes were proposed in this pull request?

`makeRootConverter` is only called with a `StructType` value. By making this method less general we can remove pattern matches, which are never actually hit outside of the test suite.

## How was this patch tested?

The existing tests.

Author: Nathan Howell <nhowell@godaddy.com>

Closes #16084 from NathanHowell/SPARK-18654.
2016-12-07 16:52:05 -08:00
Michael Armbrust 70b2bf717d [SPARK-18754][SS] Rename recentProgresses to recentProgress
Based on an informal survey, users find this option easier to understand / remember.

Author: Michael Armbrust <michael@databricks.com>

Closes #16182 from marmbrus/renameRecentProgress.
2016-12-07 15:36:29 -08:00
Shixiong Zhu edc87e1892 [SPARK-18588][TESTS] Fix flaky test: KafkaSourceStressForDontFailOnDataLossSuite
## What changes were proposed in this pull request?

Fixed the following failures:

```
org.scalatest.exceptions.TestFailedDueToTimeoutException: The code passed to eventually never returned normally. Attempted 3745 times over 1.0000790851666665 minutes. Last failure message: assertion failed: failOnDataLoss-0 not deleted after timeout.
```

```
sbt.ForkMain$ForkError: org.apache.spark.sql.streaming.StreamingQueryException: Query query-66 terminated with exception: null
	at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches(StreamExecution.scala:252)
	at org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:146)
Caused by: sbt.ForkMain$ForkError: java.lang.NullPointerException: null
	at java.util.ArrayList.addAll(ArrayList.java:577)
	at org.apache.kafka.clients.Metadata.getClusterForCurrentTopics(Metadata.java:257)
	at org.apache.kafka.clients.Metadata.update(Metadata.java:177)
	at org.apache.kafka.clients.NetworkClient$DefaultMetadataUpdater.handleResponse(NetworkClient.java:605)
	at org.apache.kafka.clients.NetworkClient$DefaultMetadataUpdater.maybeHandleCompletedReceive(NetworkClient.java:582)
	at org.apache.kafka.clients.NetworkClient.handleCompletedReceives(NetworkClient.java:450)
	at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:269)
	at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.clientPoll(ConsumerNetworkClient.java:360)
	at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:224)
	at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:192)
	at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.awaitPendingRequests(ConsumerNetworkClient.java:260)
	at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureActiveGroup(AbstractCoordinator.java:222)
	at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.ensurePartitionAssignment(ConsumerCoordinator.java:366)
	at org.apache.kafka.clients.consumer.KafkaConsumer.pollOnce(KafkaConsumer.java:978)
	at org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:938)
	at
...
```

## How was this patch tested?

Tested in #16048 by running many times.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16109 from zsxwing/fix-kafka-flaky-test.
2016-12-07 13:47:44 -08:00
sarutak bb94f61a7a [SPARK-18762][WEBUI] Web UI should be http:4040 instead of https:4040
## What changes were proposed in this pull request?

When SSL is enabled, the Spark shell shows:
```
Spark context Web UI available at https://192.168.99.1:4040
```
This is wrong because 4040 is http, not https. It redirects to the https port.
More importantly, this introduces several broken links in the UI. For example, in the master UI, the worker link is https:8081 instead of http:8081 or https:8481.

CC: mengxr liancheng

I manually tested accessing by accessing MasterPage, WorkerPage and HistoryServer with SSL enabled.

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

Closes #16190 from sarutak/SPARK-18761.
2016-12-07 11:41:23 -08:00
Shixiong Zhu dbf3e298a1 [SPARK-18764][CORE] Add a warning log when skipping a corrupted file
## What changes were proposed in this pull request?

It's better to add a warning log when skipping a corrupted file. It will be helpful when we want to finish the job first, then find them in the log and fix these files.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16192 from zsxwing/SPARK-18764.
2016-12-07 10:30:05 -08:00
Andrew Ray f1fca81b16 [SPARK-17760][SQL] AnalysisException with dataframe pivot when groupBy column is not attribute
## What changes were proposed in this pull request?

Fixes AnalysisException for pivot queries that have group by columns that are expressions and not attributes by substituting the expressions output attribute in the second aggregation and final projection.

## How was this patch tested?

existing and additional unit tests

Author: Andrew Ray <ray.andrew@gmail.com>

Closes #16177 from aray/SPARK-17760.
2016-12-07 04:44:14 -08:00
Jie Xiong c496d03b52 [SPARK-18208][SHUFFLE] Executor OOM due to a growing LongArray in BytesToBytesMap
## What changes were proposed in this pull request?

BytesToBytesMap currently does not release the in-memory storage (the longArray variable) after it spills to disk. This is typically not a problem during aggregation because the longArray should be much smaller than the pages, and because we grow the longArray at a conservative rate.

However this can lead to an OOM when an already running task is allocated more than its fair share, this can happen because of a scheduling delay. In this case the longArray can grow beyond the fair share of memory for the task. This becomes problematic when the task spills and the long array is not freed, that causes subsequent memory allocation requests to be denied by the memory manager resulting in an OOM.

This PR fixes this issuing by freeing the longArray when the BytesToBytesMap spills.

## How was this patch tested?

Existing tests and tested on realworld workloads.

Author: Jie Xiong <jiexiong@fb.com>
Author: jiexiong <jiexiong@gmail.com>

Closes #15722 from jiexiong/jie_oom_fix.
2016-12-07 04:33:30 -08:00
Sean Owen 79f5f281bb
[SPARK-18678][ML] Skewed reservoir sampling in SamplingUtils
## What changes were proposed in this pull request?

Fix reservoir sampling bias for small k. An off-by-one error meant that the probability of replacement was slightly too high -- k/(l-1) after l element instead of k/l, which matters for small k.

## How was this patch tested?

Existing test plus new test case.

Author: Sean Owen <sowen@cloudera.com>

Closes #16129 from srowen/SPARK-18678.
2016-12-07 17:34:45 +08:00
actuaryzhang b828027139
[SPARK-18701][ML] Fix Poisson GLM failure due to wrong initialization
Poisson GLM fails for many standard data sets (see example in test or JIRA). The issue is incorrect initialization leading to almost zero probability and weights. Specifically, the mean is initialized as the response, which could be zero. Applying the log link results in very negative numbers (protected against -Inf), which again leads to close to zero probability and weights in the weighted least squares. Fix and test are included in the commits.

## What changes were proposed in this pull request?
Update initialization in Poisson GLM

## How was this patch tested?
Add test in GeneralizedLinearRegressionSuite

srowen sethah yanboliang HyukjinKwon mengxr

Author: actuaryzhang <actuaryzhang10@gmail.com>

Closes #16131 from actuaryzhang/master.
2016-12-07 16:37:25 +08:00
Yanbo Liang 90b59d1bf2 [SPARK-18686][SPARKR][ML] Several cleanup and improvements for spark.logit.
## What changes were proposed in this pull request?
Several cleanup and improvements for ```spark.logit```:
* ```summary``` should return coefficients matrix, and should output labels for each class if the model is multinomial logistic regression model.
* ```summary``` should not return ```areaUnderROC, roc, pr, ...```, since most of them are DataFrame which are less important for R users. Meanwhile, these metrics ignore instance weights (setting all to 1.0) which will be changed in later Spark version. In case it will introduce breaking changes, we do not expose them currently.
* SparkR test improvement: comparing the training result with native R glmnet.
* Remove argument ```aggregationDepth``` from ```spark.logit```, since it's an expert Param(related with Spark architecture and job execution) that would be used rarely by R users.

## How was this patch tested?
Unit tests.

The ```summary``` output after this change:
multinomial logistic regression:
```
> df <- suppressWarnings(createDataFrame(iris))
> model <- spark.logit(df, Species ~ ., regParam = 0.5)
> summary(model)
$coefficients
             versicolor  virginica   setosa
(Intercept)  1.514031    -2.609108   1.095077
Sepal_Length 0.02511006  0.2649821   -0.2900921
Sepal_Width  -0.5291215  -0.02016446 0.549286
Petal_Length 0.03647411  0.1544119   -0.190886
Petal_Width  0.000236092 0.4195804   -0.4198165
```
binomial logistic regression:
```
> df <- suppressWarnings(createDataFrame(iris))
> training <- df[df$Species %in% c("versicolor", "virginica"), ]
> model <- spark.logit(training, Species ~ ., regParam = 0.5)
> summary(model)
$coefficients
             Estimate
(Intercept)  -6.053815
Sepal_Length 0.2449379
Sepal_Width  0.1648321
Petal_Length 0.4730718
Petal_Width  1.031947
```

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #16117 from yanboliang/spark-18686.
2016-12-07 00:31:11 -08:00
Tathagata Das 5c6bcdbda4 [SPARK-18671][SS][TEST-MAVEN] Follow up PR to fix test for Maven
## What changes were proposed in this pull request?

Maven compilation seem to not allow resource is sql/test to be easily referred to in kafka-0-10-sql tests. So moved the kafka-source-offset-version-2.1.0 from sql test resources to kafka-0-10-sql test resources.

## How was this patch tested?

Manually ran maven test

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

Closes #16183 from tdas/SPARK-18671-1.
2016-12-06 21:51:38 -08:00
Reynold Xin 08d6441278 Closes stale & invalid pull requests.
Closes #14537.
Closes #16181.
Closes #8318.
Closes #6848.
Closes #7265.
Closes #9543.
2016-12-06 21:06:04 -08:00
c-sahuja 01c7c6b884 Update Spark documentation to provide information on how to create External Table
## What changes were proposed in this pull request?
Although, currently, the saveAsTable does not provide an API to save the table as an external table from a DataFrame, we can achieve this functionality by using options on DataFrameWriter where the key for the map is the String: "path" and the value is another String which is the location of the external table itself. This can be provided before the call to saveAsTable is performed.

## How was this patch tested?
Documentation was reviewed for formatting and content after the push was performed on the branch.
![updated documentation](https://cloud.githubusercontent.com/assets/15376052/20953147/4cfcf308-bc57-11e6-807c-e21fb774a760.PNG)

Author: c-sahuja <sahuja@cloudera.com>

Closes #16185 from c-sahuja/createExternalTable.
2016-12-06 19:03:23 -08:00