Commit graph

21464 commits

Author SHA1 Message Date
Herman van Hovell e29b08add9 [SPARK-23208][SQL] Fix code generation for complex create array (related) expressions
## What changes were proposed in this pull request?
The `GenArrayData.genCodeToCreateArrayData` produces illegal java code when code splitting is enabled. This is used in `CreateArray` and `CreateMap` expressions for complex object arrays.

This issue is caused by a typo.

## How was this patch tested?
Added a regression test in `complexTypesSuite`.

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

Closes #20391 from hvanhovell/SPARK-23208.
2018-01-25 16:40:41 +08:00
zhoukang 45b4bbfddc [SPARK-23129][CORE] Make deserializeStream of DiskMapIterator init lazily
## What changes were proposed in this pull request?

Currently,the deserializeStream in ExternalAppendOnlyMap#DiskMapIterator init when DiskMapIterator instance created.This will cause memory use overhead when ExternalAppendOnlyMap spill too much times.

We can avoid this by making deserializeStream init when it is used the first time.
This patch make deserializeStream init lazily.

## How was this patch tested?

Exist tests

Author: zhoukang <zhoukang199191@gmail.com>

Closes #20292 from caneGuy/zhoukang/lay-diskmapiterator.
2018-01-25 15:24:52 +08:00
caoxuewen 6f0ba8472d [MINOR][SQL] add new unit test to LimitPushdown
## What changes were proposed in this pull request?

This PR is repaired as follows
1、update y -> x in "left outer join" test case ,maybe is mistake.
2、add a new test case:"left outer join and left sides are limited"
3、add a new test case:"left outer join and right sides are limited"
4、add a new test case: "right outer join and right sides are limited"
5、add a new test case: "right outer join and left sides are limited"
6、Remove annotations without code implementation

## How was this patch tested?

add new unit test case.

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

Closes #20381 from heary-cao/LimitPushdownSuite.
2018-01-24 13:06:09 -08:00
Dongjoon Hyun bc9641d902 [SPARK-23198][SS][TEST] Fix KafkaContinuousSourceStressForDontFailOnDataLossSuite to test ContinuousExecution
## What changes were proposed in this pull request?

Currently, `KafkaContinuousSourceStressForDontFailOnDataLossSuite` runs on `MicroBatchExecution`. It should test `ContinuousExecution`.

## How was this patch tested?

Pass the updated test suite.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20374 from dongjoon-hyun/SPARK-23198.
2018-01-24 12:58:44 -08:00
Mark Petruska 0e178e1523 [SPARK-22297][CORE TESTS] Flaky test: BlockManagerSuite "Shuffle registration timeout and maxAttempts conf"
## What changes were proposed in this pull request?

[Ticket](https://issues.apache.org/jira/browse/SPARK-22297)
- one of the tests seems to produce unreliable results due to execution speed variability

Since the original test was trying to connect to the test server with `40 ms` timeout, and the test server replied after `50 ms`, the error might be produced under the following conditions:
- it might occur that the test server replies correctly after `50 ms`
- but the client does only receive the timeout after `51 ms`s
- this might happen if the executor has to schedule a big number of threads, and decides to delay the thread/actor that is responsible to watch the timeout, because of high CPU load
- running an entire test suite usually produces high loads on the CPU executing the tests

## How was this patch tested?

The test's check cases remain the same and the set-up emulates the previous version's.

Author: Mark Petruska <petruska.mark@gmail.com>

Closes #19671 from mpetruska/SPARK-22297.
2018-01-24 10:25:14 -08:00
Matthew Tovbin 840dea64ab [SPARK-23152][ML] - Correctly guard against empty datasets
## What changes were proposed in this pull request?

Correctly guard against empty datasets in `org.apache.spark.ml.classification.Classifier`

## How was this patch tested?

existing tests

Author: Matthew Tovbin <mtovbin@salesforce.com>

Closes #20321 from tovbinm/SPARK-23152.
2018-01-24 13:13:44 -05:00
zuotingbing bbb87b350d [SPARK-22837][SQL] Session timeout checker does not work in SessionManager.
## What changes were proposed in this pull request?

Currently we do not call the `super.init(hiveConf)` in `SparkSQLSessionManager.init`. So we do not load the config `HIVE_SERVER2_SESSION_CHECK_INTERVAL HIVE_SERVER2_IDLE_SESSION_TIMEOUT HIVE_SERVER2_IDLE_SESSION_CHECK_OPERATION` , which cause the session timeout checker does not work.

## How was this patch tested?

manual tests

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

Closes #20025 from zuotingbing/SPARK-22837.
2018-01-24 10:07:24 -08:00
Takuya UESHIN 8c273b4162 [SPARK-23020][CORE][FOLLOWUP] Fix Java style check issues.
## What changes were proposed in this pull request?

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

```
[ERROR] src/test/java/org/apache/spark/launcher/BaseSuite.java:[21,8] (imports) UnusedImports: Unused import - java.util.concurrent.TimeUnit.
[ERROR] src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java:[27,8] (imports) UnusedImports: Unused import - java.util.concurrent.TimeUnit.
```

## How was this patch tested?

Checked manually in my local environment.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #20376 from ueshin/issues/SPARK-23020/fup1.
2018-01-24 10:00:42 -08:00
Felix Cheung e18d6f5326 [SPARK-20906][SPARKR] Add API doc example for Constrained Logistic Regression
## What changes were proposed in this pull request?

doc only changes

## How was this patch tested?

manual

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #20380 from felixcheung/rclrdoc.
2018-01-24 09:37:54 -08:00
“attilapiros” 0ec95bb7df [SPARK-22577][CORE] executor page blacklist status should update with TaskSet level blacklisting
## What changes were proposed in this pull request?

In this PR stage blacklisting is propagated to UI by introducing a new Spark listener event (SparkListenerExecutorBlacklistedForStage) which indicates the executor is blacklisted for a stage. Either because of the number of failures are exceeded a limit given for an executor (spark.blacklist.stage.maxFailedTasksPerExecutor) or because of the whole node is blacklisted for a stage (spark.blacklist.stage.maxFailedExecutorsPerNode). In case of the node is blacklisting all executors will listed as blacklisted for the stage.

Blacklisting state for a selected stage can be seen "Aggregated Metrics by Executor" table's blacklisting column, where after this change three possible labels could be found:
- "for application": when the executor is blacklisted for the application (see the configuration spark.blacklist.application.maxFailedTasksPerExecutor for details)
- "for stage": when the executor is **only** blacklisted for the stage
- "false" : when the executor is not blacklisted at all

## How was this patch tested?

It is tested both manually and with unit tests.

#### Unit tests

- HistoryServerSuite
- TaskSetBlacklistSuite
- AppStatusListenerSuite

#### Manual test for executor blacklisting

Running Spark as a local cluster:
```
$ bin/spark-shell --master "local-cluster[2,1,1024]" --conf "spark.blacklist.enabled=true" --conf "spark.blacklist.stage.maxFailedTasksPerExecutor=1" --conf "spark.blacklist.application.maxFailedTasksPerExecutor=10" --conf "spark.eventLog.enabled=true"
```

Executing:
``` scala
import org.apache.spark.SparkEnv

sc.parallelize(1 to 10, 10).map { x =>
  if (SparkEnv.get.executorId == "0") throw new RuntimeException("Bad executor")
  else (x % 3, x)
}.reduceByKey((a, b) => a + b).collect()
```

To see result check the "Aggregated Metrics by Executor" section at the bottom of picture:

![UI screenshot for stage level blacklisting executor](https://issues.apache.org/jira/secure/attachment/12905283/stage_blacklisting.png)

#### Manual test for node blacklisting

Running Spark as on a cluster:

``` bash
./bin/spark-shell --master yarn --deploy-mode client --executor-memory=2G --num-executors=8 --conf "spark.blacklist.enabled=true" --conf "spark.blacklist.stage.maxFailedTasksPerExecutor=1" --conf "spark.blacklist.stage.maxFailedExecutorsPerNode=1"  --conf "spark.blacklist.application.maxFailedTasksPerExecutor=10" --conf "spark.eventLog.enabled=true"
```

And the job was:

``` scala
import org.apache.spark.SparkEnv

sc.parallelize(1 to 10000, 10).map { x =>
  if (SparkEnv.get.executorId.toInt >= 4) throw new RuntimeException("Bad executor")
    else (x % 3, x)
}.reduceByKey((a, b) => a + b).collect()
```

The result is:

![UI screenshot for stage level node blacklisting](https://issues.apache.org/jira/secure/attachment/12906833/node_blacklisting_for_stage.png)

Here you can see apiros3.gce.test.com was node blacklisted for the stage because of failures on executor 4 and 5. As expected executor 3 is also blacklisted even it has no failures itself but sharing the node with 4 and 5.

Author: “attilapiros” <piros.attila.zsolt@gmail.com>
Author: Attila Zsolt Piros <2017933+attilapiros@users.noreply.github.com>

Closes #20203 from attilapiros/SPARK-22577.
2018-01-24 11:34:59 -06:00
Henry Robinson de36f65d3a [SPARK-23148][SQL] Allow pathnames with special characters for CSV / JSON / text
…JSON / text

## What changes were proposed in this pull request?

Fix for JSON and CSV data sources when file names include characters
that would be changed by URL encoding.

## How was this patch tested?

New unit tests for JSON, CSV and text suites

Author: Henry Robinson <henry@cloudera.com>

Closes #20355 from henryr/spark-23148.
2018-01-24 21:19:09 +09:00
Rekha Joshi 7af1a325da [SPARK-23174][BUILD][PYTHON] python code style checker update
## What changes were proposed in this pull request?
Referencing latest python code style checking from PyPi/pycodestyle
Removed pending TODO
For now, in tox.ini excluded the additional style error discovered on existing python due to latest style checker (will fallback on review comment to finalize exclusion or fix py)
Any further code styling requirement needs to be part of pycodestyle, not in SPARK.

## How was this patch tested?
./dev/run-tests

Author: Rekha Joshi <rekhajoshm@gmail.com>
Author: rjoshi2 <rekhajoshm@gmail.com>

Closes #20338 from rekhajoshm/SPARK-11222.
2018-01-24 21:13:47 +09:00
gatorsmile 4e7b49041a Revert "[SPARK-23195][SQL] Keep the Hint of Cached Data"
This reverts commit 44cc4daf3a.
2018-01-23 22:38:20 -08:00
neilalex f54b65c15a [SPARK-21727][R] Allow multi-element atomic vector as column type in SparkR DataFrame
## What changes were proposed in this pull request?

A fix to https://issues.apache.org/jira/browse/SPARK-21727, "Operating on an ArrayType in a SparkR DataFrame throws error"

## How was this patch tested?

- Ran tests at R\pkg\tests\run-all.R (see below attached results)
- Tested the following lines in SparkR, which now seem to execute without error:

```
indices <- 1:4
myDf <- data.frame(indices)
myDf$data <- list(rep(0, 20))
mySparkDf <- as.DataFrame(myDf)
collect(mySparkDf)
```

[2018-01-22 SPARK-21727 Test Results.txt](https://github.com/apache/spark/files/1653535/2018-01-22.SPARK-21727.Test.Results.txt)

felixcheung yanboliang sun-rui shivaram

_The contribution is my original work and I license the work to the project under the project’s open source license_

Author: neilalex <neil@neilalex.com>

Closes #20352 from neilalex/neilalex-sparkr-arraytype.
2018-01-23 22:31:14 -08:00
Liang-Chi Hsieh a3911cf896 [SPARK-23177][SQL][PYSPARK] Extract zero-parameter UDFs from aggregate
## What changes were proposed in this pull request?

We extract Python UDFs in logical aggregate which depends on aggregate expression or grouping key in ExtractPythonUDFFromAggregate rule. But Python UDFs which don't depend on above expressions should also be extracted to avoid the issue reported in the JIRA.

A small code snippet to reproduce that issue looks like:
```python
import pyspark.sql.functions as f

df = spark.createDataFrame([(1,2), (3,4)])
f_udf = f.udf(lambda: str("const_str"))
df2 = df.distinct().withColumn("a", f_udf())
df2.show()
```

Error exception is raised as:
```
: org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: pythonUDF0#50
        at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:56)
        at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:91)
        at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:90)
        at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:267)
        at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:267)
        at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
        at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:266)
        at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272)
        at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272)
        at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306)
        at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187)
        at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304)
        at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272)
        at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:256)
        at org.apache.spark.sql.catalyst.expressions.BindReferences$.bindReference(BoundAttribute.scala:90)
        at org.apache.spark.sql.execution.aggregate.HashAggregateExec$$anonfun$38.apply(HashAggregateExec.scala:514)
        at org.apache.spark.sql.execution.aggregate.HashAggregateExec$$anonfun$38.apply(HashAggregateExec.scala:513)
```

This exception raises because `HashAggregateExec` tries to bind the aliased Python UDF expression (e.g., `pythonUDF0#50 AS a#44`) to grouping key.

## How was this patch tested?

Added test.

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

Closes #20360 from viirya/SPARK-23177.
2018-01-24 11:43:48 +09:00
Tathagata Das 15adcc8273 [SPARK-23197][DSTREAMS] Increased timeouts to resolve flakiness
## What changes were proposed in this pull request?

Increased timeout from 50 ms to 300 ms (50 ms was really too low).

## How was this patch tested?
Multiple rounds of tests.

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

Closes #20371 from tdas/SPARK-23197.
2018-01-23 16:24:20 -08:00
gatorsmile 44cc4daf3a [SPARK-23195][SQL] Keep the Hint of Cached Data
## What changes were proposed in this pull request?
The broadcast hint of the cached plan is lost if we cache the plan. This PR is to correct it.

```Scala
  val df1 = spark.createDataFrame(Seq((1, "4"), (2, "2"))).toDF("key", "value")
  val df2 = spark.createDataFrame(Seq((1, "1"), (2, "2"))).toDF("key", "value")
  broadcast(df2).cache()
  df2.collect()
  val df3 = df1.join(df2, Seq("key"), "inner")
```

## How was this patch tested?
Added a test.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20368 from gatorsmile/cachedBroadcastHint.
2018-01-23 16:17:09 -08:00
gatorsmile 613c290336 [SPARK-23192][SQL] Keep the Hint after Using Cached Data
## What changes were proposed in this pull request?

The hint of the plan segment is lost, if the plan segment is replaced by the cached data.

```Scala
      val df1 = spark.createDataFrame(Seq((1, "4"), (2, "2"))).toDF("key", "value")
      val df2 = spark.createDataFrame(Seq((1, "1"), (2, "2"))).toDF("key", "value")
      df2.cache()
      val df3 = df1.join(broadcast(df2), Seq("key"), "inner")
```

This PR is to fix it.

## How was this patch tested?
Added a test

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20365 from gatorsmile/fixBroadcastHintloss.
2018-01-23 14:56:28 -08:00
Bago Amirbekian 05839d1648 [SPARK-22735][ML][DOC] Added VectorSizeHint docs and examples.
## What changes were proposed in this pull request?

Added documentation for new transformer.

Author: Bago Amirbekian <bago@databricks.com>

Closes #20285 from MrBago/sizeHintDocs.
2018-01-23 14:11:23 -08:00
Marcelo Vanzin dc4761fd8f [SPARK-17088][HIVE] Fix 'sharesHadoopClasses' option when creating client.
Because the call to the constructor of HiveClientImpl crosses class loader
boundaries, different versions of the same class (Configuration in this
case) were loaded, and that caused a runtime error when instantiating the
client. By using a safer type in the signature of the constructor, it's
possible to avoid the problem.

I considered removing 'sharesHadoopClasses', but it may still be desired
(even though there are 0 users of it since it was not working). When Spark
starts to support Hadoop 3, it may be necessary to use that option to
load clients for older Hive metastore versions that don't know about
Hadoop 3.

Tested with added unit test.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20169 from vanzin/SPARK-17088.
2018-01-23 12:51:40 -08:00
Marcelo Vanzin bdebb8e48e [SPARK-20664][SPARK-23103][CORE] Follow-up: remove workaround for .
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20353 from vanzin/SPARK-20664.
2018-01-23 10:12:13 -08:00
gatorsmile ee572ba8c1 [SPARK-20749][SQL][FOLLOW-UP] Override prettyName for bit_length and octet_length
## What changes were proposed in this pull request?
We need to override the prettyName for bit_length and octet_length for getting the expected auto-generated alias name.

## How was this patch tested?
The existing tests

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20358 from gatorsmile/test2.3More.
2018-01-23 21:36:20 +09:00
Xingbo Jiang 96cb60bc33 [SPARK-22465][FOLLOWUP] Update the number of partitions of default partitioner when defaultParallelism is set
## What changes were proposed in this pull request?

#20002 purposed a way to safe check the default partitioner, however, if `spark.default.parallelism` is set, the defaultParallelism still could be smaller than the proper number of partitions for upstreams RDDs. This PR tries to extend the approach to address the condition when `spark.default.parallelism` is set.

The requirements where the PR helps with are :
- Max partitioner is not eligible since it is atleast an order smaller, and
- User has explicitly set 'spark.default.parallelism', and
- Value of 'spark.default.parallelism' is lower than max partitioner
- Since max partitioner was discarded due to being at least an order smaller, default parallelism is worse - even though user specified.

Under the rest cases, the changes should be no-op.

## How was this patch tested?

Add corresponding test cases in `PairRDDFunctionsSuite` and `PartitioningSuite`.

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

Closes #20091 from jiangxb1987/partitioner.
2018-01-23 04:08:32 -08:00
Li Jin b2ce17b4c9 [SPARK-22274][PYTHON][SQL] User-defined aggregation functions with pandas udf (full shuffle)
## What changes were proposed in this pull request?

Add support for using pandas UDFs with groupby().agg().

This PR introduces a new type of pandas UDF - group aggregate pandas UDF. This type of UDF defines a transformation of multiple pandas Series -> a scalar value. Group aggregate pandas UDFs can be used with groupby().agg(). Note group aggregate pandas UDF doesn't support partial aggregation, i.e., a full shuffle is required.

This PR doesn't support group aggregate pandas UDFs that return ArrayType, StructType or MapType. Support for these types is left for future PR.

## How was this patch tested?

GroupbyAggPandasUDFTests

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

Closes #19872 from icexelloss/SPARK-22274-groupby-agg.
2018-01-23 14:11:30 +09:00
Wenchen Fan 51eb750263 [SPARK-22389][SQL] data source v2 partitioning reporting interface
## What changes were proposed in this pull request?

a new interface which allows data source to report partitioning and avoid shuffle at Spark side.

The design is pretty like the internal distribution/partitioing framework. Spark defines a `Distribution` interfaces and several concrete implementations, and ask the data source to report a `Partitioning`, the `Partitioning` should tell Spark if it can satisfy a `Distribution` or not.

## How was this patch tested?

new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20201 from cloud-fan/partition-reporting.
2018-01-22 15:21:09 -08:00
Jacek Laskowski 76b8b840dd [MINOR] Typo fixes
## What changes were proposed in this pull request?

Typo fixes

## How was this patch tested?

Local build / Doc-only changes

Author: Jacek Laskowski <jacek@japila.pl>

Closes #20344 from jaceklaskowski/typo-fixes.
2018-01-22 13:55:14 -06:00
Sandor Murakozi 446948af1d [SPARK-23121][CORE] Fix for ui becoming unaccessible for long running streaming apps
## What changes were proposed in this pull request?

The allJobs and the job pages attempt to use stage attempt and DAG visualization from the store, but for long running jobs they are not guaranteed to be retained, leading to exceptions when these pages are rendered.

To fix it `store.lastStageAttempt(stageId)` and `store.operationGraphForJob(jobId)` are wrapped in `store.asOption` and default values are used if the info is missing.

## How was this patch tested?

Manual testing of the UI, also using the test command reported in SPARK-23121:

./bin/spark-submit --class org.apache.spark.examples.streaming.HdfsWordCount ./examples/jars/spark-examples_2.11-2.4.0-SNAPSHOT.jar /spark

Closes #20287

Author: Sandor Murakozi <smurakozi@gmail.com>

Closes #20330 from smurakozi/SPARK-23121.
2018-01-22 10:36:28 -08:00
Rekha Joshi 4327ccf289 [SPARK-11630][CORE] ClosureCleaner moved from warning to debug
## What changes were proposed in this pull request?
ClosureCleaner moved from warning to debug
## How was this patch tested?
Existing tests

Author: Rekha Joshi <rekhajoshm@gmail.com>
Author: rjoshi2 <rekhajoshm@gmail.com>

Closes #20337 from rekhajoshm/SPARK-11630-1.
2018-01-22 08:36:17 -06:00
hyukjinkwon 87ffe7addd [SPARK-7721][PYTHON][TESTS] Adds PySpark coverage generation script
## What changes were proposed in this pull request?

Note that this PR was made based on the top of https://github.com/apache/spark/pull/20151. So, it almost leaves the main codes intact.

This PR proposes to add a script for the preparation of automatic PySpark coverage generation. Now, it's difficult to check the actual coverage in case of PySpark. With this script, it allows to run tests by the way we did via `run-tests` script before. The usage is exactly the same with `run-tests` script as this basically wraps it.

This script and PR alone should also be useful. I was asked about how to run this before, and seems some reviewers (including me) need this. It would be also useful to run it manually.

It usually requires a small diff in normal Python projects but PySpark cases are a bit different because apparently we are unable to track the coverage after it's forked. So, here, I made a custom worker that forces the coverage, based on the top of https://github.com/apache/spark/pull/20151.

I made a simple demo. Please take a look - https://spark-test.github.io/pyspark-coverage-site.

To show up the structure, this PR adds the files as below:

```
python
├── .coveragerc  # Runtime configuration when we run the script.
├── run-tests-with-coverage  # The script that has coverage support and wraps run-tests script.
└── test_coverage  # Directories that have files required when running coverage.
    ├── conf
    │   └── spark-defaults.conf  # Having the configuration 'spark.python.daemon.module'.
    ├── coverage_daemon.py  # A daemon having custom fix and wrapping our daemon.py
    └── sitecustomize.py  # Initiate coverage with COVERAGE_PROCESS_START
```

Note that this PR has a minor nit:

[This scope](04e44b37cc/python/pyspark/daemon.py (L148-L169)) in `daemon.py` is not in the coverage results as basically I am producing the coverage results in `worker.py` separately and then merging it. I believe it's not a big deal.

In a followup, I might have a site that has a single up-to-date PySpark coverage from the master branch as the fallback / default, or have a site that has multiple PySpark coverages and the site link will be left to each pull request.

## How was this patch tested?

Manually tested. Usage is the same with the existing Python test script - `./python/run-tests`. For example,

```
sh run-tests-with-coverage --python-executables=python3 --modules=pyspark-sql
```

Running this will generate HTMLs under `./python/test_coverage/htmlcov`.

Console output example:

```
sh run-tests-with-coverage --python-executables=python3,python --modules=pyspark-core
Running PySpark tests. Output is in /.../spark/python/unit-tests.log
Will test against the following Python executables: ['python3', 'python']
Will test the following Python modules: ['pyspark-core']
Starting test(python): pyspark.tests
Starting test(python3): pyspark.tests
...
Tests passed in 231 seconds
Combining collected coverage data under /.../spark/python/test_coverage/coverage_data
Reporting the coverage data at /...spark/python/test_coverage/coverage_data/coverage
Name                         Stmts   Miss Branch BrPart  Cover
--------------------------------------------------------------
pyspark/__init__.py             41      0      8      2    96%
...
pyspark/profiler.py             74     11     22      5    83%
pyspark/rdd.py                 871     40    303     32    93%
pyspark/rddsampler.py           68     10     32      2    82%
...
--------------------------------------------------------------
TOTAL                         8521   3077   2748    191    59%
Generating HTML files for PySpark coverage under /.../spark/python/test_coverage/htmlcov
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20204 from HyukjinKwon/python-coverage.
2018-01-22 22:12:50 +09:00
Wenchen Fan 5d680cae48 [SPARK-23090][SQL] polish ColumnVector
## What changes were proposed in this pull request?

Several improvements:
* provide a default implementation for the batch get methods
* rename `getChildColumn` to `getChild`, which is more concise
* remove `getStruct(int, int)`, it's only used to simplify the codegen, which is an internal thing, we should not add a public API for this purpose.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20277 from cloud-fan/column-vector.
2018-01-22 20:56:38 +08:00
gatorsmile 896e45af5f [MINOR][SQL][TEST] Test case cleanups for recent PRs
## What changes were proposed in this pull request?
Revert the unneeded test case changes we made in SPARK-23000

Also fixes the test suites that do not call `super.afterAll()` in the local `afterAll`. The `afterAll()` of `TestHiveSingleton` actually reset the environments.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20341 from gatorsmile/testRelated.
2018-01-22 04:32:59 -08:00
gatorsmile 78801881c4 [SPARK-23170][SQL] Dump the statistics of effective runs of analyzer and optimizer rules
## What changes were proposed in this pull request?

Dump the statistics of effective runs of analyzer and optimizer rules.

## How was this patch tested?

Do a manual run of TPCDSQuerySuite

```
=== Metrics of Analyzer/Optimizer Rules ===
Total number of runs: 175899
Total time: 25.486559948 seconds

Rule                                                                                               Effective Time / Total Time                     Effective Runs / Total Runs

org.apache.spark.sql.catalyst.optimizer.ColumnPruning                                              1603280450 / 2868461549                         761 / 1877
org.apache.spark.sql.catalyst.analysis.Analyzer$CTESubstitution                                    2045860009 / 2056602674                         37 / 788
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAggregateFunctions                          440719059 / 1693110949                          38 / 1982
org.apache.spark.sql.catalyst.optimizer.Optimizer$OptimizeSubqueries                               1429834919 / 1446016225                         39 / 285
org.apache.spark.sql.catalyst.optimizer.PruneFilters                                               33273083 / 1389586938                           3 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences                                  821183615 / 1266668754                          616 / 1982
org.apache.spark.sql.catalyst.optimizer.ReorderJoin                                                775837028 / 866238225                           132 / 1592
org.apache.spark.sql.catalyst.analysis.DecimalPrecision                                            550683593 / 748854507                           211 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveSubquery                                    513075345 / 634370596                           49 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$FixNullability                                     33475731 / 606406532                            12 / 742
org.apache.spark.sql.catalyst.analysis.TypeCoercion$ImplicitTypeCasts                              193144298 / 545403925                           86 / 1982
org.apache.spark.sql.catalyst.optimizer.BooleanSimplification                                      18651497 / 495725004                            7 / 1592
org.apache.spark.sql.catalyst.optimizer.PushPredicateThroughJoin                                   369257217 / 489934378                           709 / 1592
org.apache.spark.sql.catalyst.optimizer.RemoveRedundantAliases                                     3707000 / 468291609                             9 / 1592
org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints                                410155900 / 435254175                           192 / 285
org.apache.spark.sql.execution.datasources.FindDataSourceTable                                     348885539 / 371855866                           233 / 1982
org.apache.spark.sql.catalyst.optimizer.NullPropagation                                            11307645 / 307531225                            26 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveFunctions                                   120324545 / 304948785                           294 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$FunctionArgumentConversion                     92323199 / 286695007                            38 / 1982
org.apache.spark.sql.catalyst.optimizer.PushDownPredicate                                          230084193 / 265845972                           785 / 1592
org.apache.spark.sql.catalyst.analysis.TypeCoercion$PromoteStrings                                 45938401 / 265144009                            40 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$InConversion                                   14888776 / 261499450                            1 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$CaseWhenCoercion                               113796384 / 244913861                           29 / 1982
org.apache.spark.sql.catalyst.optimizer.ConstantFolding                                            65008069 / 236548480                            126 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ExtractGenerator                                   0 / 226338929                                   0 / 1982
org.apache.spark.sql.catalyst.analysis.ResolveTimeZone                                             98134906 / 221323770                            417 / 1982
org.apache.spark.sql.catalyst.optimizer.ReorderAssociativeOperator                                 0 / 208421703                                   0 / 1592
org.apache.spark.sql.catalyst.optimizer.OptimizeIn                                                 8762534 / 199351958                             16 / 1592
org.apache.spark.sql.catalyst.analysis.TypeCoercion$DateTimeOperations                             11980016 / 190779046                            27 / 1982
org.apache.spark.sql.catalyst.optimizer.SimplifyBinaryComparison                                   0 / 188887385                                   0 / 1592
org.apache.spark.sql.catalyst.optimizer.SimplifyConditionals                                       0 / 186812106                                   0 / 1592
org.apache.spark.sql.catalyst.optimizer.SimplifyCaseConversionExpressions                          0 / 183885230                                   0 / 1592
org.apache.spark.sql.catalyst.optimizer.SimplifyCasts                                              17128295 / 182901910                            69 / 1592
org.apache.spark.sql.catalyst.analysis.TypeCoercion$Division                                       14579110 / 180309340                            8 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$BooleanEquality                                0 / 176740516                                   0 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$IfCoercion                                     0 / 170781986                                   0 / 1982
org.apache.spark.sql.catalyst.optimizer.LikeSimplification                                         771605 / 164136736                              1 / 1592
org.apache.spark.sql.catalyst.optimizer.RemoveDispensableExpressions                               0 / 155958962                                   0 / 1592
org.apache.spark.sql.catalyst.analysis.ResolveCreateNamedStruct                                    0 / 151222943                                   0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveWindowOrder                                 7534632 / 146596355                             14 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$EltCoercion                                    0 / 144488654                                   0 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$ConcatCoercion                                 0 / 142403338                                   0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveWindowFrame                                 12067635 / 141500665                            21 / 1982
org.apache.spark.sql.catalyst.analysis.TimeWindowing                                               0 / 140431958                                   0 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$WindowFrameCoercion                            0 / 125471960                                   0 / 1982
org.apache.spark.sql.catalyst.optimizer.EliminateOuterJoin                                         14226972 / 124922019                            11 / 1592
org.apache.spark.sql.catalyst.analysis.TypeCoercion$StackCoercion                                  0 / 123613887                                   0 / 1982
org.apache.spark.sql.catalyst.optimizer.RewriteCorrelatedScalarSubquery                            8491071 / 121179056                             7 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveGroupingAnalytics                           55526073 / 120290529                            11 / 1982
org.apache.spark.sql.catalyst.optimizer.ConstantPropagation                                        0 / 113886790                                   0 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveDeserializer                                52383759 / 107160222                            148 / 1982
org.apache.spark.sql.catalyst.analysis.CleanupAliases                                              52543524 / 102091518                            344 / 1086
org.apache.spark.sql.catalyst.optimizer.RemoveRedundantProject                                     40682895 / 94403652                             342 / 1877
org.apache.spark.sql.catalyst.analysis.Analyzer$ExtractWindowExpressions                           38473816 / 89740578                             23 / 1982
org.apache.spark.sql.catalyst.optimizer.CollapseProject                                            46806090 / 83315506                             281 / 1877
org.apache.spark.sql.catalyst.optimizer.FoldablePropagation                                        0 / 78750087                                    0 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAliases                                     13742765 / 77227258                             47 / 1982
org.apache.spark.sql.catalyst.optimizer.CombineFilters                                             53386729 / 76960344                             448 / 1592
org.apache.spark.sql.execution.datasources.DataSourceAnalysis                                      68034341 / 75724186                             24 / 742
org.apache.spark.sql.catalyst.analysis.Analyzer$LookupFunctions                                    0 / 71151084                                    0 / 750
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveMissingReferences                           12139848 / 67599140                             8 / 1982
org.apache.spark.sql.catalyst.optimizer.PullupCorrelatedPredicates                                 45017938 / 65968777                             23 / 285
org.apache.spark.sql.execution.datasources.v2.PushDownOperatorsToDataSource                        0 / 60937767                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.CollapseRepartition                                        0 / 59897237                                    0 / 1592
org.apache.spark.sql.catalyst.optimizer.PushProjectionThroughUnion                                 8547262 / 53941370                              10 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$HandleNullInputsForUDF                             0 / 52735976                                    0 / 742
org.apache.spark.sql.catalyst.analysis.TypeCoercion$WidenSetOperationTypes                         9797713 / 52401665                              9 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$PullOutNondeterministic                            0 / 51741500                                    0 / 742
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations                                   28614911 / 51061186                             233 / 1990
org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions                               0 / 50621510                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.CombineUnions                                              2777800 / 50262112                              17 / 1877
org.apache.spark.sql.catalyst.analysis.Analyzer$GlobalAggregates                                   1640641 / 49633909                              46 / 1982
org.apache.spark.sql.catalyst.optimizer.DecimalAggregates                                          20198374 / 48488419                             100 / 385
org.apache.spark.sql.catalyst.optimizer.LimitPushDown                                              0 / 45052523                                    0 / 1592
org.apache.spark.sql.catalyst.optimizer.CombineLimits                                              0 / 44719443                                    0 / 1592
org.apache.spark.sql.catalyst.optimizer.EliminateSorts                                             0 / 44216930                                    0 / 1592
org.apache.spark.sql.catalyst.optimizer.RewritePredicateSubquery                                   36235699 / 44165786                             148 / 285
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveNewInstance                                 0 / 42750307                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveUpCast                                      0 / 41811748                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveOrdinalInOrderByAndGroupBy                  3819476 / 41776562                              4 / 1982
org.apache.spark.sql.catalyst.optimizer.ComputeCurrentTime                                         0 / 40527808                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.CollapseWindow                                             0 / 36832538                                    0 / 1592
org.apache.spark.sql.catalyst.optimizer.EliminateSerialization                                     0 / 36120667                                    0 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAggAliasInGroupBy                           0 / 32435826                                    0 / 1982
org.apache.spark.sql.execution.datasources.PreprocessTableCreation                                 0 / 32145218                                    0 / 742
org.apache.spark.sql.execution.datasources.ResolveSQLOnFile                                        0 / 30295614                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolvePivot                                       0 / 30111655                                    0 / 1982
org.apache.spark.sql.catalyst.expressions.codegen.package$ExpressionCanonicalizer$CleanExpressions 59930 / 28038201                                26 / 8280
org.apache.spark.sql.catalyst.analysis.ResolveInlineTables                                         0 / 27808108                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveSubqueryColumnAliases                       0 / 27066690                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveGenerate                                    0 / 26660210                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveNaturalAndUsingJoin                         0 / 25255184                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.ResolveTableValuedFunctions                                 0 / 24663088                                    0 / 1990
org.apache.spark.sql.catalyst.analysis.SubstituteUnresolvedOrdinals                                9709079 / 24450670                              4 / 788
org.apache.spark.sql.catalyst.analysis.ResolveHints$ResolveBroadcastHints                          0 / 23776535                                    0 / 750
org.apache.spark.sql.catalyst.optimizer.ReplaceExpressions                                         0 / 22697895                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.CheckCartesianProducts                                     0 / 22523798                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.ReplaceDistinctWithAggregate                               988593 / 21535410                               15 / 300
org.apache.spark.sql.catalyst.optimizer.EliminateMapObjects                                        0 / 20269996                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.RewriteDistinctAggregates                                  0 / 19388592                                    0 / 285
org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases                                    17675532 / 18971185                             215 / 285
org.apache.spark.sql.catalyst.optimizer.GetCurrentDatabase                                         0 / 18271152                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.PropagateEmptyRelation                                     2077097 / 17190855                              3 / 288
org.apache.spark.sql.catalyst.analysis.EliminateBarriers                                           0 / 16736359                                    0 / 1086
org.apache.spark.sql.execution.OptimizeMetadataOnlyQuery                                           0 / 16669341                                    0 / 285
org.apache.spark.sql.catalyst.analysis.UpdateOuterReferences                                       0 / 14470235                                    0 / 742
org.apache.spark.sql.catalyst.optimizer.ReplaceExceptWithAntiJoin                                  6715625 / 12190561                              1 / 300
org.apache.spark.sql.catalyst.optimizer.ReplaceIntersectWithSemiJoin                               3451793 / 11431432                              7 / 300
org.apache.spark.sql.execution.python.ExtractPythonUDFFromAggregate                                0 / 10810568                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.RemoveRepetitionFromGroupExpressions                       344198 / 10475276                               1 / 286
org.apache.spark.sql.catalyst.analysis.Analyzer$WindowsSubstitution                                0 / 10386630                                    0 / 788
org.apache.spark.sql.catalyst.analysis.EliminateUnions                                             0 / 10096526                                    0 / 788
org.apache.spark.sql.catalyst.analysis.AliasViewChild                                              0 / 9991706                                     0 / 742
org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation                                     0 / 9649334                                     0 / 288
org.apache.spark.sql.catalyst.analysis.ResolveHints$RemoveAllHints                                 0 / 8739109                                     0 / 750
org.apache.spark.sql.execution.datasources.PreprocessTableInsertion                                0 / 8420889                                     0 / 742
org.apache.spark.sql.catalyst.analysis.EliminateView                                               0 / 8319134                                     0 / 285
org.apache.spark.sql.catalyst.optimizer.RemoveLiteralFromGroupExpressions                          0 / 7392627                                     0 / 286
org.apache.spark.sql.catalyst.optimizer.ReplaceExceptWithFilter                                    0 / 7170516                                     0 / 300
org.apache.spark.sql.catalyst.optimizer.SimplifyCreateArrayOps                                     0 / 7109643                                     0 / 1592
org.apache.spark.sql.catalyst.optimizer.SimplifyCreateStructOps                                    0 / 6837590                                     0 / 1592
org.apache.spark.sql.catalyst.optimizer.SimplifyCreateMapOps                                       0 / 6617848                                     0 / 1592
org.apache.spark.sql.catalyst.optimizer.CombineConcats                                             0 / 5768406                                     0 / 1592
org.apache.spark.sql.catalyst.optimizer.ReplaceDeduplicateWithAggregate                            0 / 5349831                                     0 / 285
org.apache.spark.sql.catalyst.optimizer.CombineTypedFilters                                        0 / 5186642                                     0 / 285
org.apache.spark.sql.catalyst.optimizer.EliminateDistinct                                          0 / 2427686                                     0 / 285
org.apache.spark.sql.catalyst.optimizer.CostBasedJoinReorder                                       0 / 2420436                                     0 / 285

```

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20342 from gatorsmile/reportExecution.
2018-01-22 04:31:24 -08:00
gatorsmile 73281161fc [SPARK-23122][PYSPARK][FOLLOW-UP] Update the docs for UDF Registration
## What changes were proposed in this pull request?

This PR is to update the docs for UDF registration

## How was this patch tested?

N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20348 from gatorsmile/testUpdateDoc.
2018-01-22 04:27:59 -08:00
Arseniy Tashoyan 60175e959f [MINOR][DOC] Fix the path to the examples jar
## What changes were proposed in this pull request?

The example jar file is now in ./examples/jars directory of Spark distribution.

Author: Arseniy Tashoyan <tashoyan@users.noreply.github.com>

Closes #20349 from tashoyan/patch-1.
2018-01-22 20:17:05 +08:00
Marcelo Vanzin ec22897615 [SPARK-23020][CORE] Fix races in launcher code, test.
The race in the code is because the handle might update
its state to the wrong state if the connection handling
thread is still processing incoming data; so the handle
needs to wait for the connection to finish up before
checking the final state.

The race in the test is because when waiting for a handle
to reach a final state, the waitFor() method needs to wait
until all handle state is updated (which also includes
waiting for the connection thread above to finish).
Otherwise, waitFor() may return too early, which would cause
a bunch of different races (like the listener not being yet
notified of the state change, or being in the middle of
being notified, or the handle not being properly disposed
and causing postChecks() to assert).

On top of that I found, by code inspection, a couple of
potential races that could make a handle end up in the
wrong state when being killed.

The original version of this fix introduced the flipped
version of the first race described above; the connection
closing might override the handle state before the
handle might have a chance to do cleanup. The fix there
is to only dispose of the handle from the connection
when there is an error, and let the handle dispose
itself in the normal case.

The fix also caused a bug in YarnClusterSuite to be surfaced;
the code was checking for a file in the classpath that was
not expected to be there in client mode. Because of the above
issues, the error was not propagating correctly and the (buggy)
test was incorrectly passing.

Tested by running the existing unit tests a lot (and not
seeing the errors I was seeing before).

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20297 from vanzin/SPARK-23020.
2018-01-22 14:49:12 +08:00
Dongjoon Hyun 8142a3b883 [MINOR][SQL] Fix wrong comments on org.apache.spark.sql.parquet.row.attributes
## What changes were proposed in this pull request?

This PR fixes the wrong comment on `org.apache.spark.sql.parquet.row.attributes`
which is useful for UDTs like Vector/Matrix. Please see [SPARK-22320](https://issues.apache.org/jira/browse/SPARK-22320) for the usage.

Originally, [SPARK-19411](bf493686eb (diff-ee26d4c4be21e92e92a02e9f16dbc285L314)) left this behind during removing optional column metadatas. In the same PR, the same comment was removed at line 310-311.

## How was this patch tested?

N/A (This is about comments).

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20346 from dongjoon-hyun/minor_comment_parquet.
2018-01-22 15:18:57 +09:00
Russell Spitzer 11daeb8332 [SPARK-22976][CORE] Cluster mode driver dir removed while running
## What changes were proposed in this pull request?

The clean up logic on the worker perviously determined the liveness of a
particular applicaiton based on whether or not it had running executors.
This would fail in the case that a directory was made for a driver
running in cluster mode if that driver had no running executors on the
same machine. To preserve driver directories we consider both executors
and running drivers when checking directory liveness.

## How was this patch tested?

Manually started up two node cluster with a single core on each node. Turned on worker directory cleanup and set the interval to 1 second and liveness to one second. Without the patch the driver directory is removed immediately after the app is launched. With the patch it is not

### Without Patch
```
INFO  2018-01-05 23:48:24,693 Logging.scala:54 - Asked to launch driver driver-20180105234824-0000
INFO  2018-01-05 23:48:25,293 Logging.scala:54 - Changing view acls to: cassandra
INFO  2018-01-05 23:48:25,293 Logging.scala:54 - Changing modify acls to: cassandra
INFO  2018-01-05 23:48:25,294 Logging.scala:54 - Changing view acls groups to:
INFO  2018-01-05 23:48:25,294 Logging.scala:54 - Changing modify acls groups to:
INFO  2018-01-05 23:48:25,294 Logging.scala:54 - SecurityManager: authentication disabled; ui acls disabled; users  with view permissions: Set(cassandra); groups with view permissions: Set(); users  with modify permissions: Set(cassandra); groups with modify permissions: Set()
INFO  2018-01-05 23:48:25,330 Logging.scala:54 - Copying user jar file:/home/automaton/writeRead-0.1.jar to /var/lib/spark/worker/driver-20180105234824-0000/writeRead-0.1.jar
INFO  2018-01-05 23:48:25,332 Logging.scala:54 - Copying /home/automaton/writeRead-0.1.jar to /var/lib/spark/worker/driver-20180105234824-0000/writeRead-0.1.jar
INFO  2018-01-05 23:48:25,361 Logging.scala:54 - Launch Command: "/usr/lib/jvm/jdk1.8.0_40//bin/java" ....
****
INFO  2018-01-05 23:48:56,577 Logging.scala:54 - Removing directory: /var/lib/spark/worker/driver-20180105234824-0000  ### << Cleaned up
****
--
One minute passes while app runs (app has 1 minute sleep built in)
--

WARN  2018-01-05 23:49:58,080 ShuffleSecretManager.java:73 - Attempted to unregister application app-20180105234831-0000 when it is not registered
INFO  2018-01-05 23:49:58,081 ExternalShuffleBlockResolver.java:163 - Application app-20180105234831-0000 removed, cleanupLocalDirs = false
INFO  2018-01-05 23:49:58,081 ExternalShuffleBlockResolver.java:163 - Application app-20180105234831-0000 removed, cleanupLocalDirs = false
INFO  2018-01-05 23:49:58,082 ExternalShuffleBlockResolver.java:163 - Application app-20180105234831-0000 removed, cleanupLocalDirs = true
INFO  2018-01-05 23:50:00,999 Logging.scala:54 - Driver driver-20180105234824-0000 exited successfully
```

With Patch
```
INFO  2018-01-08 23:19:54,603 Logging.scala:54 - Asked to launch driver driver-20180108231954-0002
INFO  2018-01-08 23:19:54,975 Logging.scala:54 - Changing view acls to: automaton
INFO  2018-01-08 23:19:54,976 Logging.scala:54 - Changing modify acls to: automaton
INFO  2018-01-08 23:19:54,976 Logging.scala:54 - Changing view acls groups to:
INFO  2018-01-08 23:19:54,976 Logging.scala:54 - Changing modify acls groups to:
INFO  2018-01-08 23:19:54,976 Logging.scala:54 - SecurityManager: authentication disabled; ui acls disabled; users  with view permissions: Set(automaton); groups with view permissions: Set(); users  with modify permissions: Set(automaton); groups with modify permissions: Set()
INFO  2018-01-08 23:19:55,029 Logging.scala:54 - Copying user jar file:/home/automaton/writeRead-0.1.jar to /var/lib/spark/worker/driver-20180108231954-0002/writeRead-0.1.jar
INFO  2018-01-08 23:19:55,031 Logging.scala:54 - Copying /home/automaton/writeRead-0.1.jar to /var/lib/spark/worker/driver-20180108231954-0002/writeRead-0.1.jar
INFO  2018-01-08 23:19:55,038 Logging.scala:54 - Launch Command: ......
INFO  2018-01-08 23:21:28,674 ShuffleSecretManager.java:69 - Unregistered shuffle secret for application app-20180108232000-0000
INFO  2018-01-08 23:21:28,675 ExternalShuffleBlockResolver.java:163 - Application app-20180108232000-0000 removed, cleanupLocalDirs = false
INFO  2018-01-08 23:21:28,675 ExternalShuffleBlockResolver.java:163 - Application app-20180108232000-0000 removed, cleanupLocalDirs = false
INFO  2018-01-08 23:21:28,681 ExternalShuffleBlockResolver.java:163 - Application app-20180108232000-0000 removed, cleanupLocalDirs = true
INFO  2018-01-08 23:21:31,703 Logging.scala:54 - Driver driver-20180108231954-0002 exited successfully
*****
INFO  2018-01-08 23:21:32,346 Logging.scala:54 - Removing directory: /var/lib/spark/worker/driver-20180108231954-0002 ### < Happening AFTER the Run completes rather than during it
*****
```

Author: Russell Spitzer <Russell.Spitzer@gmail.com>

Closes #20298 from RussellSpitzer/SPARK-22976-master.
2018-01-22 12:27:51 +08:00
王晓哲 602c6d82d8 [SPARK-20947][PYTHON] Fix encoding/decoding error in pipe action
## What changes were proposed in this pull request?

Pipe action convert objects into strings using a way that was affected by the default encoding setting of Python environment.

This patch fixed the problem. The detailed description is added here:

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

## How was this patch tested?

Run the following statement in pyspark-shell, and it will NOT raise exception if this patch is applied:

```python
sc.parallelize([u'\u6d4b\u8bd5']).pipe('cat').collect()
```

Author: 王晓哲 <wxz@linkdoc.com>

Closes #18277 from chaoslawful/fix_pipe_encoding_error.
2018-01-22 10:43:12 +09:00
hyukjinkwon 12faae295e [SPARK-23169][INFRA][R] Run lintr on the changes of lint-r script and .lintr configuration
## What changes were proposed in this pull request?

When running the `run-tests` script, seems we don't run lintr on the changes of `lint-r` script and `.lintr` configuration.

## How was this patch tested?

Jenkins builds

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20339 from HyukjinKwon/check-r-changed.
2018-01-22 09:45:27 +09:00
Felix Cheung 2239d7a410 [SPARK-21293][SS][SPARKR] Add doc example for streaming join, dedup
## What changes were proposed in this pull request?

streaming programming guide changes

## How was this patch tested?

manually

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #20340 from felixcheung/rstreamdoc.
2018-01-21 11:23:51 -08:00
Marco Gaido 4f43d27c9e [SPARK-22119][ML] Add cosine distance to KMeans
## What changes were proposed in this pull request?

Currently, KMeans assumes the only possible distance measure to be used is the Euclidean. This PR aims to add the cosine distance support to the KMeans algorithm.

## How was this patch tested?

existing and added UTs.

Author: Marco Gaido <marcogaido91@gmail.com>
Author: Marco Gaido <mgaido@hortonworks.com>

Closes #19340 from mgaido91/SPARK-22119.
2018-01-21 08:51:12 -06:00
Marco Gaido 121dc96f08 [SPARK-23087][SQL] CheckCartesianProduct too restrictive when condition is false/null
## What changes were proposed in this pull request?

CheckCartesianProduct raises an AnalysisException also when the join condition is always false/null. In this case, we shouldn't raise it, since the result will not be a cartesian product.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20333 from mgaido91/SPARK-23087.
2018-01-20 22:39:49 -08:00
fjh100456 00d169156d [SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' and 'spark.sql.orc.compression.codec' configuration doesn't take effect on hive table writing
[SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' and 'spark.sql.orc.compression.codec' configuration doesn't take effect on hive table writing

What changes were proposed in this pull request?

Pass ‘spark.sql.parquet.compression.codec’ value to ‘parquet.compression’.
Pass ‘spark.sql.orc.compression.codec’ value to ‘orc.compress’.

How was this patch tested?

Add test.

Note:
This is the same issue mentioned in #19218 . That branch was deleted mistakenly, so make a new pr instead.

gatorsmile maropu dongjoon-hyun discipleforteen

Author: fjh100456 <fu.jinhua6@zte.com.cn>
Author: Takeshi Yamamuro <yamamuro@apache.org>
Author: Wenchen Fan <wenchen@databricks.com>
Author: gatorsmile <gatorsmile@gmail.com>
Author: Yinan Li <liyinan926@gmail.com>
Author: Marcelo Vanzin <vanzin@cloudera.com>
Author: Juliusz Sompolski <julek@databricks.com>
Author: Felix Cheung <felixcheung_m@hotmail.com>
Author: jerryshao <sshao@hortonworks.com>
Author: Li Jin <ice.xelloss@gmail.com>
Author: Gera Shegalov <gera@apache.org>
Author: chetkhatri <ckhatrimanjal@gmail.com>
Author: Joseph K. Bradley <joseph@databricks.com>
Author: Bago Amirbekian <bago@databricks.com>
Author: Xianjin YE <advancedxy@gmail.com>
Author: Bruce Robbins <bersprockets@gmail.com>
Author: zuotingbing <zuo.tingbing9@zte.com.cn>
Author: Kent Yao <yaooqinn@hotmail.com>
Author: hyukjinkwon <gurwls223@gmail.com>
Author: Adrian Ionescu <adrian@databricks.com>

Closes #20087 from fjh100456/HiveTableWriting.
2018-01-20 14:49:49 -08:00
Shashwat Anand 84a076e0e9 [SPARK-23165][DOC] Spelling mistake fix in quick-start doc.
## What changes were proposed in this pull request?

Fix spelling in quick-start doc.

## How was this patch tested?

Doc only.

Author: Shashwat Anand <me@shashwat.me>

Closes #20336 from ashashwat/SPARK-23165.
2018-01-20 14:34:37 -08:00
Sean Owen 396cdfbea4 [SPARK-23091][ML] Incorrect unit test for approxQuantile
## What changes were proposed in this pull request?

Narrow bound on approx quantile test to epsilon from 2*epsilon to match paper

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #20324 from srowen/SPARK-23091.
2018-01-19 22:46:34 -08:00
Kent Yao 793841c6b8 [SPARK-21771][SQL] remove useless hive client in SparkSQLEnv
## What changes were proposed in this pull request?

Once a meta hive client is created, it generates its SessionState which creates a lot of session related directories, some deleteOnExit, some does not. if a hive client is useless we may not create it at the very start.

## How was this patch tested?
N/A

cc hvanhovell cloud-fan

Author: Kent Yao <11215016@zju.edu.cn>

Closes #18983 from yaooqinn/patch-1.
2018-01-19 15:49:29 -08:00
Marcelo Vanzin f6da41b015 [SPARK-23135][UI] Fix rendering of accumulators in the stage page.
This follows the behavior of 2.2: only named accumulators with a
value are rendered.

Screenshot:
![accs](https://user-images.githubusercontent.com/1694083/35065700-df409114-fb82-11e7-87c1-550c3f674371.png)

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20299 from vanzin/SPARK-23135.
2018-01-19 13:14:24 -08:00
Marcelo Vanzin aa3a1276f9 [SPARK-23103][CORE] Ensure correct sort order for negative values in LevelDB.
The code was sorting "0" as "less than" negative values, which is a little
wrong. Fix is simple, most of the changes are the added test and related
cleanup.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20284 from vanzin/SPARK-23103.
2018-01-19 13:32:20 -06:00
Marcelo Vanzin fed2139f05 [SPARK-20664][CORE] Delete stale application data from SHS.
Detect the deletion of event log files from storage, and remove
data about the related application attempt in the SHS.

Also contains code to fix SPARK-21571 based on code by ericvandenbergfb.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20138 from vanzin/SPARK-20664.
2018-01-19 13:26:37 -06:00
Marcelo Vanzin 07296a61c2 [INFRA] Close stale PR.
Closes #20185.
2018-01-19 10:25:18 -08:00