Commit graph

23621 commits

Author SHA1 Message Date
Gengliang Wang f5b9370da2 [SPARK-26709][SQL] OptimizeMetadataOnlyQuery does not handle empty records correctly
## What changes were proposed in this pull request?

When reading from empty tables, the optimization `OptimizeMetadataOnlyQuery` may return wrong results:
```
sql("CREATE TABLE t (col1 INT, p1 INT) USING PARQUET PARTITIONED BY (p1)")
sql("INSERT INTO TABLE t PARTITION (p1 = 5) SELECT ID FROM range(1, 1)")
sql("SELECT MAX(p1) FROM t")
```
The result is supposed to be `null`. However, with the optimization the result is `5`.

The rule is originally ported from https://issues.apache.org/jira/browse/HIVE-1003 in #13494. In Hive, the rule is disabled by default in a later release(https://issues.apache.org/jira/browse/HIVE-15397), due to the same problem.

It is hard to completely avoid the correctness issue. Because data sources like Parquet can be metadata-only. Spark can't tell whether it is empty or not without actually reading it. This PR disable the optimization by default.

## How was this patch tested?

Unit test

Closes #23635 from gengliangwang/optimizeMetadata.

Lead-authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Co-authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-24 18:24:49 -08:00
Hyukjin Kwon d2ff10cbe1 [SPARK-23674][ML] Adds Spark ML Events to Instrumentation
## What changes were proposed in this pull request?

This PR proposes to add ML events to Instrumentation, and use it in Pipeline so that other developers can track and add some actions for them.

## Introduction

ML events (like SQL events) can be quite useful when people want to track and make some actions for corresponding ML operations. For instance, I have been working on integrating
Apache Spark with [Apache Atlas](https://atlas.apache.org/QuickStart.html). With some custom changes with this PR, I can visualise ML pipeline as below:

![spark_ml_streaming_lineage](https://user-images.githubusercontent.com/6477701/49682779-394bca80-faf5-11e8-85b8-5fae28b784b3.png)

Another good thing that might have to be considered is, that we can interact this with other SQL/Streaming events. For instance, where the input `Dataset` is originated. For instance, with current Apache Spark, I can visualise SQL operations as below:

![screen shot 2018-12-10 at 9 41 36 am](https://user-images.githubusercontent.com/6477701/49706269-d9bdfe00-fc5f-11e8-943a-3309d1856ba5.png)

I think we can combine those existing lineages together to easily understand where the data comes and goes. Currently, ML side is a hole so the lineages can't be connected for the current Apache Spark ..

To add up, I think it's not to mention how useful it is to track the SQL/Streaming operations. Likewise, I would like to propose ML events as well (as lowest stability `Unstable` APIs for now - no guarantee about stability).

## Implementation Details

### Sends event (but not expose ML specific listener)

**`mllib/src/main/scala/org/apache/spark/ml/events.scala`**

```scala
Unstable
case class ...StartEvent(caller, input)
Unstable
case class ...EndEvent(caller, output)

trait MLEvents {
  // Wrappers to send events:
  // def with...Event(body) = {
  //   body()
  //   SparkContext.getOrCreate().listenerBus.post(event)
  // }
}
```

This trait is used by `Instrumentation`.

```scala
class Instrumentation ... with MLEvents {
```

and used as below:

```scala
instrumented { instr =>
  instr.with...Event(...) {
    ...
  }
}
```

This way mimics both:

**1. Catalog events (see `org/apache/spark/sql/catalyst/catalog/events.scala`)**

- This allows a Catalog specific listener to be added `ExternalCatalogEventListener`

- It's implemented in a way of wrapping whole `ExternalCatalog` named `ExternalCatalogWithListener`
which delegates the operations to `ExternalCatalog`

This is not quite possible in this case because most of instances (like `Pipeline`) will be directly created in most of cases. We might be able to do that via extending `ListenerBus` for all possible instances but IMHO it's too invasive. Also, exposing another ML specific listener sounds a bit too much at this stage. Therefore, I simply borrowed file name and structures here

**2. SQL execution events (see `org/apache/spark/sql/execution/SQLExecution.scala`)**

- Add an object that wraps a body to send events

Current apporach is rather close to this. It has a `with...` wrapper to send events. I borrowed this approach to be consistent.

## Usage

It needs a custom implementation for a query listener. For instance,

with the custom listener below:

```scala
class CustomMLListener extends SparkListener
  def onOtherEvents(e) = e match {
    case e: MLEvent => // do something
    case _ => // pass
  }
}
```

There are two (existing) ways to use this.

```scala
spark.sparkContext.addSparkListener(new CustomMLListener)
```

```bash
spark-submit ...\
  --conf spark.extraListeners=CustomMLListener\
  ...
```

It's also similar with other existing implementation in SQL side.

## Target users

1. I think someone in general would likely utilise this feature like other event listeners. At least, I can see some interests going on outside.

    - SQL Listener
      - https://stackoverflow.com/questions/46409339/spark-listener-to-an-sql-query
      - http://apache-spark-user-list.1001560.n3.nabble.com/spark-sql-Custom-Query-Execution-listener-via-conf-properties-td30979.html

    - Streaming Query Listener
      - https://jhui.github.io/2017/01/15/Apache-Spark-Streaming/
      -  http://apache-spark-developers-list.1001551.n3.nabble.com/Structured-Streaming-with-Watermark-td25413.html#a25416

2. Someone would likely run this via Atlas. The plugin mirror intentionally is exposed at [spark-atlas-connector](https://github.com/hortonworks-spark/spark-atlas-connector) so that anyone could do something about lineage and governance in Atlas. I'm trying to show integrated lineages in Apache Spark but this is a missing hole.

## How was this patch tested?

Manually tested and unit tests were added.

Closes #23263 from HyukjinKwon/SPARK-23674-1.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-25 10:11:49 +08:00
Hyukjin Kwon d97481ba17 [MINOR] Add Jenkins and AppVeyor badges at README.md
## What changes were proposed in this pull request?

This PR adds Jenkins build and AppVeyor build badges

Take a look at cc24b9e0b3/README.md

## How was this patch tested?

Manually tested by `grip`.

Closes #23629 from HyukjinKwon/minor-badges.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-25 10:04:21 +08:00
Ilya Matiach b2d36f65db [SPARK-19591][ML][MLLIB] Add sample weights to decision trees
This is updated PR https://github.com/apache/spark/pull/16722 to latest master

## What changes were proposed in this pull request?

This patch adds support for sample weights to DecisionTreeRegressor and DecisionTreeClassifier.

Note: This patch does not add support for sample weights to RandomForest. As discussed in the JIRA, we would like to add sample weights into the bagging process. This patch is large enough as is, and there are some additional considerations to be made for random forests. Since the machinery introduced here needs to be present regardless, I have opted to leave random forests for a follow up pr.
## How was this patch tested?

The algorithms are tested to ensure that:
    1. Arbitrary scaling of constant weights has no effect
    2. Outliers with small weights do not affect the learned model
    3. Oversampling and weighting are equivalent

Unit tests are also added to test other smaller components.
## Summary of changes

   - Impurity aggregators now store weighted sufficient statistics. They also store a raw count, however, since this is needed to use minInstancesPerNode.

   - Impurity aggregators now also hold the raw count.

   - This patch maintains the meaning of minInstancesPerNode, in that the parameter still corresponds to raw, unweighted counts. It also adds a new parameter minWeightFractionPerNode which requires that nodes must contain at least minWeightFractionPerNode * weightedNumExamples total weight.

   - This patch modifies findSplitsForContinuousFeatures to use weighted sums. Unit tests are added.

   - TreePoint is modified to hold a sample weight

   - BaggedPoint is modified from:
``` Scala
private[spark] class BaggedPoint[Datum](val datum: Datum, val subsampleWeights: Array[Double]) extends Serializable
```
to
``` Scala
private[spark] class BaggedPoint[Datum](
    val datum: Datum,
    val subsampleCounts: Array[Int],
    val sampleWeight: Double) extends Serializable
```
We do not simply multiply the counts by the weight and store that because we need the raw counts and the weight in order to use both minInstancesPerNode and minWeightPerNode

**Note**: many of the changed files are due simply to using Instance instead of LabeledPoint

Closes #21632 from imatiach-msft/ilmat/sample-weights.

Authored-by: Ilya Matiach <ilmat@microsoft.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-24 18:20:28 -07:00
Imran Rashid 3699763fda [SPARK-26697][CORE] Log local & remote block sizes.
## What changes were proposed in this pull request?

To help debugging failed or slow tasks, its really useful to know the
size of the blocks getting fetched.  Though that is available at the
debug level, debug logs aren't on in general -- but there is already an
info level log line that this augments a little.

## How was this patch tested?

Ran very basic local-cluster mode app, looked at logs.  Example line:

```
INFO ShuffleBlockFetcherIterator: Getting 2 (194.0 B) non-empty blocks including 1 (97.0 B) local blocks and 1 (97.0 B) remote blocks
```

Full suite via jenkins.

Closes #23621 from squito/SPARK-26697.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-01-24 16:13:58 -08:00
Liupengcheng 8d667c511c [SPARK-26530][CORE] Validate heartheat arguments in HeartbeatReceiver
## What changes were proposed in this pull request?

Currently, heartbeat related arguments is not validated in spark, so if these args are inproperly specified, the Application may run for a while and not failed until the max executor failures reached(especially with spark.dynamicAllocation.enabled=true), thus may incurs resources waste.

This PR is to precheck these arguments in HeartbeatReceiver to fix this problem.

## How was this patch tested?

NA-just validation changes

Closes #23445 from liupc/validate-heartbeat-arguments-in-SparkSubmitArguments.

Authored-by: Liupengcheng <liupengcheng@xiaomi.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-01-24 15:12:57 -08:00
Rob Vesse 69dab94b13 [SPARK-26687][K8S] Fix handling of custom Dockerfile paths
## What changes were proposed in this pull request?

With the changes from vanzin's PR #23019 (SPARK-26025) we use a pared down temporary Docker build context which significantly improves build times.  However the way this is implemented leads to non-intuitive behaviour when supplying custom Docker file paths.  This is because of the following code snippets:

```
(cd $(img_ctx_dir base) && docker build $NOCACHEARG "${BUILD_ARGS[]}" \
    -t $(image_ref spark) \
    -f "$BASEDOCKERFILE" .)
```

Since the script changes to the temporary build context directory and then runs `docker build` there any path given for the Docker file is taken as relative to the temporary build context directory rather than to the directory where the user invoked the script.  This is rather unintuitive and produces somewhat unhelpful errors e.g.

```
> ./bin/docker-image-tool.sh -r rvesse -t badpath -p resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile build
Sending build context to Docker daemon  218.4MB
Step 1/15 : FROM openjdk:8-alpine
 ---> 5801f7d008e5
Step 2/15 : ARG spark_uid=185
 ---> Using cache
 ---> 5fd63df1ca39
...
Successfully tagged rvesse/spark:badpath
unable to prepare context: unable to evaluate symlinks in Dockerfile path: lstat /Users/rvesse/Documents/Work/Code/spark/target/tmp/docker/pyspark/resource-managers: no such file or directory
Failed to build PySpark Docker image, please refer to Docker build output for details.
```

Here we can see that the relative path that was valid where the user typed the command was not valid inside the build context directory.

To resolve this we need to ensure that we are resolving relative paths to Docker files appropriately which we do by adding a `resolve_file` function to the script and invoking that on the supplied Docker file paths

## How was this patch tested?

Validated that relative paths now work as expected:

```
> ./bin/docker-image-tool.sh -r rvesse -t badpath -p resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile build
Sending build context to Docker daemon  218.4MB
Step 1/15 : FROM openjdk:8-alpine
 ---> 5801f7d008e5
Step 2/15 : ARG spark_uid=185
 ---> Using cache
 ---> 5fd63df1ca39
Step 3/15 : RUN set -ex &&     apk upgrade --no-cache &&     apk add --no-cache bash tini libc6-compat linux-pam krb5 krb5-libs &&     mkdir -p /opt/spark &&     mkdir -p /opt/spark/examples &&     mkdir -p /opt/spark/work-dir &&     touch /opt/spark/RELEASE &&     rm /bin/sh &&     ln -sv /bin/bash /bin/sh &&     echo "auth required pam_wheel.so use_uid" >> /etc/pam.d/su &&     chgrp root /etc/passwd && chmod ug+rw /etc/passwd
 ---> Using cache
 ---> eb0a568e032f
Step 4/15 : COPY jars /opt/spark/jars
...
Successfully tagged rvesse/spark:badpath
Sending build context to Docker daemon  6.599MB
Step 1/13 : ARG base_img
Step 2/13 : ARG spark_uid=185
Step 3/13 : FROM $base_img
 ---> 8f4fff16f903
Step 4/13 : WORKDIR /
 ---> Running in 25466e66f27f
Removing intermediate container 25466e66f27f
 ---> 1470b6efae61
Step 5/13 : USER 0
 ---> Running in b094b739df37
Removing intermediate container b094b739df37
 ---> 6a27eb4acad3
Step 6/13 : RUN mkdir ${SPARK_HOME}/python
 ---> Running in bc8002c5b17c
Removing intermediate container bc8002c5b17c
 ---> 19bb12f4286a
Step 7/13 : RUN apk add --no-cache python &&     apk add --no-cache python3 &&     python -m ensurepip &&     python3 -m ensurepip &&     rm -r /usr/lib/python*/ensurepip &&     pip install --upgrade pip setuptools &&     rm -r /root/.cache
 ---> Running in 12dcba5e527f
...
Successfully tagged rvesse/spark-py:badpath
```

Closes #23613 from rvesse/SPARK-26687.

Authored-by: Rob Vesse <rvesse@dotnetrdf.org>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-01-24 10:11:55 -08:00
Tom van Bussel 9813b1d074 [SPARK-26690] Track query execution and time cost for checkpoints
## What changes were proposed in this pull request?

Checkpoints of Dataframes currently do not show up in SQL UI. This PR fixes that by setting an execution id for the execution of the checkpoint by wrapping the checkpoint code with a `withAction`.

## How was this patch tested?

A unit test was added to DatasetSuite.

Closes #23636 from tomvanbussel/SPARK-26690.

Authored-by: Tom van Bussel <tom.vanbussel@databricks.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
2019-01-24 16:44:39 +01:00
Bruce Robbins d4a30fa9af [SPARK-26680][SQL] Eagerly create inputVars while conditions are appropriate
## What changes were proposed in this pull request?

When a user passes a Stream to groupBy, ```CodegenSupport.consume``` ends up lazily generating ```inputVars``` from a Stream, since the field ```output``` will be a Stream. At the time ```output.zipWithIndex.map``` is called, conditions are correct. However, by the time the map operation actually executes, conditions are no longer appropriate. The closure used by the map operation ends up using a reference to the partially created ```inputVars```. As a result, a StackOverflowError occurs.

This PR ensures that ```inputVars``` is eagerly created while conditions are appropriate. It seems this was also an issue with the code path for creating ```inputVars``` from ```outputVars``` (SPARK-25767). I simply extended the solution for that code path to encompass both code paths.

## How was this patch tested?

SQL unit tests
new test
python tests

Closes #23617 from bersprockets/SPARK-26680_opt1.

Authored-by: Bruce Robbins <bersprockets@gmail.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
2019-01-24 11:18:08 +01:00
Ryan Blue d5a97c1c2c [SPARK-26682][SQL] Use taskAttemptID instead of attemptNumber for Hadoop.
## What changes were proposed in this pull request?

Updates the attempt ID used by FileFormatWriter. Tasks in stage attempts use the same task attempt number and could conflict. Using Spark's task attempt ID guarantees that Hadoop TaskAttemptID instances are unique.

## How was this patch tested?

Existing tests. Also validated that we no longer detect this failure case in our logs after deployment.

Closes #23608 from rdblue/SPARK-26682-fix-hadoop-task-attempt-id.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-24 12:45:25 +08:00
Dave DeCaprio d0e9219e03 [SPARK-26617][SQL] Cache manager locks
## What changes were proposed in this pull request?

Fixed several places in CacheManager where a write lock was being held while running the query optimizer.  This could cause a very lock block if the query optimization takes a long time.  This builds on changes from [SPARK-26548] that fixed this issue for one specific case in the CacheManager.

gatorsmile This is very similar to the PR you approved last week.

## How was this patch tested?

Has been tested on a live system where the blocking was causing major issues and it is working well.
CacheManager has no explicit unit test but is used in many places internally as part of the SharedState.

Closes #23539 from DaveDeCaprio/cache-manager-locks.

Lead-authored-by: Dave DeCaprio <daved@alum.mit.edu>
Co-authored-by: David DeCaprio <daved@alum.mit.edu>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-24 10:48:48 +08:00
ayudovin 11be22bb5e [SPARK-25713][SQL] implementing copy for ColumnArray
## What changes were proposed in this pull request?

Implement copy() for ColumnarArray

## How was this patch tested?
 Updating test case to existing tests in ColumnVectorSuite

Closes #23569 from ayudovin/copy-for-columnArray.

Authored-by: ayudovin <a.yudovin6695@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-24 10:35:44 +08:00
Anton Okolnychyi 0df29bfbdc
[SPARK-26706][SQL] Fix illegalNumericPrecedence for ByteType
## What changes were proposed in this pull request?

This PR contains a minor change in `Cast$mayTruncate` that fixes its logic for bytes.

Right now, `mayTruncate(ByteType, LongType)` returns `false` while `mayTruncate(ShortType, LongType)` returns `true`. Consequently, `spark.range(1, 3).as[Byte]` and `spark.range(1, 3).as[Short]` behave differently.

Potentially, this bug can silently corrupt someone's data.
```scala
// executes silently even though Long is converted into Byte
spark.range(Long.MaxValue - 10, Long.MaxValue).as[Byte]
  .map(b => b - 1)
  .show()
+-----+
|value|
+-----+
|  -12|
|  -11|
|  -10|
|   -9|
|   -8|
|   -7|
|   -6|
|   -5|
|   -4|
|   -3|
+-----+
// throws an AnalysisException: Cannot up cast `id` from bigint to smallint as it may truncate
spark.range(Long.MaxValue - 10, Long.MaxValue).as[Short]
  .map(s => s - 1)
  .show()
```
## How was this patch tested?

This PR comes with a set of unit tests.

Closes #23632 from aokolnychyi/cast-fix.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2019-01-24 00:12:26 +00:00
Liupengcheng 0446363ef4 [SPARK-26660] Add warning logs when broadcasting large task binary
## What changes were proposed in this pull request?

Currently, some ML library may generate large ml model, which may be referenced in the task closure, so driver will broadcasting large task binary, and executor may not able to deserialize it and result in OOM failures(for instance, executor's memory is not enough). This problem not only affects apps using ml library, some user specified closure or function which refers large data may also have this problem.

In order to facilitate the debuging of memory problem caused by large taskBinary broadcast, we can add same warning logs for it.

This PR will add some warning logs on the driver side when broadcasting a large task binary, and it also included some minor log changes in the reading of broadcast.

## How was this patch tested?
NA-Just log changes.

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

Closes #23580 from liupc/Add-warning-logs-for-large-taskBinary-size.

Authored-by: Liupengcheng <liupengcheng@xiaomi.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-23 08:51:39 -06:00
Ryan Blue d008e23ab5 [SPARK-26681][SQL] Support Ammonite inner-class scopes.
## What changes were proposed in this pull request?

This adds a new pattern to recognize Ammonite REPL classes and return the correct scope.

## How was this patch tested?

Manually tested with Spark in an Ammonite session.

Closes #23607 from rdblue/SPARK-26681-support-ammonite-scopes.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-23 08:50:03 -06:00
Maxim Gekk 46d5bb9a0f [SPARK-26653][SQL] Use Proleptic Gregorian calendar in parsing JDBC lower/upper bounds
## What changes were proposed in this pull request?

In the PR, I propose using of the `stringToDate` and `stringToTimestamp` methods in parsing JDBC lower/upper bounds of the partition column if it has `DateType` or `TimestampType`. Since those methods have been ported on Proleptic Gregorian calendar by #23512, the PR switches parsing of JDBC bounds of the partition column on the calendar as well.

## How was this patch tested?

This was tested by `JDBCSuite`.

Closes #23597 from MaxGekk/jdbc-parse-timestamp-bounds.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-23 20:23:17 +08:00
Takeshi Yamamuro 1ed1b4d8e1 [SPARK-26637][SQL] Makes GetArrayItem nullability more precise
## What changes were proposed in this pull request?
In the master, GetArrayItem nullable is always true;
cf133e6110/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala (L236)

But, If input array size is constant and ordinal is foldable, we could make GetArrayItem nullability more precise. This pr added code to make `GetArrayItem` nullability more precise.

## How was this patch tested?
Added tests in `ComplexTypeSuite`.

Closes #23566 from maropu/GetArrayItemNullability.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-23 15:33:02 +08:00
Ngone51 3da71f2da1 [SPARK-22465][CORE][FOLLOWUP] Use existing partitioner when defaultNumPartitions is equal to maxPartitioner.numPartitions
## What changes were proposed in this pull request?

Followup of #20091. We could also use existing partitioner when defaultNumPartitions is equal to the maxPartitioner's numPartitions.

## How was this patch tested?

Existed.

Closes #23581 from Ngone51/dev-use-existing-partitioner-when-defaultNumPartitions-equalTo-MaxPartitioner#-numPartitions.

Authored-by: Ngone51 <ngone_5451@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-23 10:23:40 +08:00
Sean Owen 6dcad38ba3 [SPARK-26228][MLLIB] OOM issue encountered when computing Gramian matrix
## What changes were proposed in this pull request?

Avoid memory problems in closure cleaning when handling large Gramians (>= 16K rows/cols) by using null as zeroValue

## How was this patch tested?

Existing tests.
Note that it's hard to test the case that triggers this issue as it would require a large amount of memory and run a while. I confirmed locally that a 16K x 16K Gramian failed with tons of driver memory before, and didn't fail upfront after this change.

Closes #23600 from srowen/SPARK-26228.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-22 19:22:06 -06:00
Shahid 0d35f9ea3a [SPARK-24484][MLLIB] Power Iteration Clustering is giving incorrect clustering results when there are mutiple leading eigen values.
## What changes were proposed in this pull request?
![image](https://user-images.githubusercontent.com/23054875/41823325-e83e1d34-781b-11e8-8c34-fc6e7a042f3f.png)

![image](https://user-images.githubusercontent.com/23054875/41823367-733c9ba4-781c-11e8-8da2-b26460c2af63.png)
![image](https://user-images.githubusercontent.com/23054875/41823409-179dd910-781d-11e8-8d8c-9865156fad15.png)

**Method to determine if the top eigen values has same magnitude but opposite signs**
The vector is written as a linear combination of the eigen vectors at iteration k.
![image](https://user-images.githubusercontent.com/23054875/41822941-f8b13d4c-7814-11e8-8091-54c02721c1c5.png)
![image](https://user-images.githubusercontent.com/23054875/41822982-b80a6fc4-7815-11e8-9129-ed96a14f037f.png)
![image](https://user-images.githubusercontent.com/23054875/41823022-5b69e906-7816-11e8-847a-8fa5f0b6200e.png)

![image](https://user-images.githubusercontent.com/23054875/41823087-54311398-7817-11e8-90bf-e1be2bbff323.png)
![image](https://user-images.githubusercontent.com/23054875/41823121-e0b78324-7817-11e8-9596-379bd2e518af.png)
![image](https://user-images.githubusercontent.com/23054875/41823151-965319d2-7818-11e8-8b91-10f6276ace62.png)
![image](https://user-images.githubusercontent.com/23054875/41823182-75cdbad6-7819-11e8-912f-23c66a8359de.png)
![image](https://user-images.githubusercontent.com/23054875/41823221-1ca77a36-781a-11e8-9a40-48bd165797cc.png)
![image](https://user-images.githubusercontent.com/23054875/41823272-f6962b2a-781a-11e8-9978-1b2dc0dc8b2c.png)
![image](https://user-images.githubusercontent.com/23054875/41823303-75b296f0-781b-11e8-8501-6133b04769c8.png)

**So, we need to check if the reileigh coefficient at the convergence is lesser than the norm of the estimated eigen vector before normalizing**

(Please fill in changes proposed in this fix)
Added a UT

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

Closes #21627 from shahidki31/picConvergence.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-22 18:29:18 -06:00
Darcy Shen 9d2a11554b [MINOR][DOC] Documentation on JVM options for SBT
## What changes were proposed in this pull request?

Documentation and .gitignore

## How was this patch tested?

Manual test that SBT honors the settings in .jvmopts if present

Closes #23615 from sadhen/impr/gitignore.

Authored-by: Darcy Shen <sadhen@zoho.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-22 18:27:24 -06:00
Kris Mok 02d8ae3d59
[SPARK-26661][SQL] Show actual class name of the writing command in CTAS explain
## What changes were proposed in this pull request?

The explain output of the Hive CTAS command, regardless of whether it's actually writing via Hive's SerDe or converted into using Spark's data source, would always show that it's using `InsertIntoHiveTable` because it's hardcoded.

e.g.
```
Execute OptimizedCreateHiveTableAsSelectCommand [Database:default, TableName: foo, InsertIntoHiveTable]
```
This CTAS is converted into using Spark's data source, but it still says `InsertIntoHiveTable` in the explain output.

It's better to show the actual class name of the writing command used. For the example above, it'd be:
```
Execute OptimizedCreateHiveTableAsSelectCommand [Database:default, TableName: foo, InsertIntoHadoopFsRelationCommand]
```

## How was this patch tested?

Added test case in `HiveExplainSuite`

Closes #23582 from rednaxelafx/fix-explain-1.

Authored-by: Kris Mok <kris.mok@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-22 13:55:41 -08:00
Rob Vesse dc2da72100 [SPARK-26685][K8S] Correct placement of ARG declaration
Latest Docker releases are stricter in their enforcement of build argument scope.  The location of the `ARG spark_uid` declaration in the Python and R Dockerfiles means the variable is out of scope by the time it is used in a `USER` declaration resulting in a container running as root rather than the default/configured UID.

Also with some of the refactoring of the script that has happened since my PR that introduced the configurable UID it turns out the `-u <uid>` argument is not being properly passed to the Python and R image builds when those are opted into

## What changes were proposed in this pull request?

This commit moves the `ARG` declaration to just before the argument is used such that it is in scope.  It also ensures that Python and R image builds receive the build arguments that include the `spark_uid` argument where relevant

## How was this patch tested?

Prior to the patch images are produced where the Python and R images ignore the default/configured UID:

```
> docker run -it --entrypoint /bin/bash rvesse/spark-py:uid456
bash-4.4# whoami
root
bash-4.4# id -u
0
bash-4.4# exit
> docker run -it --entrypoint /bin/bash rvesse/spark:uid456
bash-4.4$ id -u
456
bash-4.4$ exit
```

Note that the Python image is still running as `root` having ignored the configured UID of 456 while the base image has the correct UID because the relevant `ARG` declaration is correctly in scope.

After the patch the correct UID is observed:

```
> docker run -it --entrypoint /bin/bash rvesse/spark-r:uid456
bash-4.4$ id -u
456
bash-4.4$ exit
exit
> docker run -it --entrypoint /bin/bash rvesse/spark-py:uid456
bash-4.4$ id -u
456
bash-4.4$ exit
exit
> docker run -it --entrypoint /bin/bash rvesse/spark:uid456
bash-4.4$ id -u
456
bash-4.4$ exit
```

Closes #23611 from rvesse/SPARK-26685.

Authored-by: Rob Vesse <rvesse@dotnetrdf.org>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-01-22 10:31:17 -08:00
Rob Vesse c542c247bb [SPARK-25887][K8S] Configurable K8S context support
This enhancement allows for specifying the desired context to use for the initial K8S client auto-configuration.  This allows users to more easily access alternative K8S contexts without having to first
explicitly change their current context via kubectl.

Explicitly set my K8S context to a context pointing to a non-existent cluster, then launched Spark jobs with explicitly specified contexts via the new `spark.kubernetes.context` configuration property.

Example Output:

```
> kubectl config current-context
minikube
> minikube status
minikube: Stopped
cluster:
kubectl:
> ./spark-submit --master k8s://https://localhost:6443 --deploy-mode cluster --name spark-pi --class org.apache.spark.examples.SparkPi --conf spark.executor.instances=2 --conf spark.kubernetes.context=docker-for-desktop --conf spark.kubernetes.container.image=rvesse/spark:debian local:///opt/spark/examples/jars/spark-examples_2.11-3.0.0-SNAPSHOT.jar 4
18/10/31 11:57:51 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
18/10/31 11:57:51 INFO SparkKubernetesClientFactory: Auto-configuring K8S client using context docker-for-desktop from users K8S config file
18/10/31 11:57:52 INFO LoggingPodStatusWatcherImpl: State changed, new state:
	 pod name: spark-pi-1540987071845-driver
	 namespace: default
	 labels: spark-app-selector -> spark-2c4abc226ed3415986eb602bd13f3582, spark-role -> driver
	 pod uid: 32462cac-dd04-11e8-b6c6-025000000001
	 creation time: 2018-10-31T11:57:52Z
	 service account name: default
	 volumes: spark-local-dir-1, spark-conf-volume, default-token-glpfv
	 node name: N/A
	 start time: N/A
	 phase: Pending
	 container status: N/A
18/10/31 11:57:52 INFO LoggingPodStatusWatcherImpl: State changed, new state:
	 pod name: spark-pi-1540987071845-driver
	 namespace: default
	 labels: spark-app-selector -> spark-2c4abc226ed3415986eb602bd13f3582, spark-role -> driver
	 pod uid: 32462cac-dd04-11e8-b6c6-025000000001
	 creation time: 2018-10-31T11:57:52Z
	 service account name: default
	 volumes: spark-local-dir-1, spark-conf-volume, default-token-glpfv
	 node name: docker-for-desktop
	 start time: N/A
	 phase: Pending
	 container status: N/A
...
18/10/31 11:58:03 INFO LoggingPodStatusWatcherImpl: State changed, new state:
	 pod name: spark-pi-1540987071845-driver
	 namespace: default
	 labels: spark-app-selector -> spark-2c4abc226ed3415986eb602bd13f3582, spark-role -> driver
	 pod uid: 32462cac-dd04-11e8-b6c6-025000000001
	 creation time: 2018-10-31T11:57:52Z
	 service account name: default
	 volumes: spark-local-dir-1, spark-conf-volume, default-token-glpfv
	 node name: docker-for-desktop
	 start time: 2018-10-31T11:57:52Z
	 phase: Succeeded
	 container status:
		 container name: spark-kubernetes-driver
		 container image: rvesse/spark:debian
		 container state: terminated
		 container started at: 2018-10-31T11:57:54Z
		 container finished at: 2018-10-31T11:58:02Z
		 exit code: 0
		 termination reason: Completed
```

Without the `spark.kubernetes.context` setting this will fail because the current context - `minikube` - is pointing to a non-running cluster e.g.

```
> ./spark-submit --master k8s://https://localhost:6443 --deploy-mode cluster --name spark-pi --class org.apache.spark.examples.SparkPi --conf spark.executor.instances=2 --conf spark.kubernetes.container.image=rvesse/spark:debian local:///opt/spark/examples/jars/spark-examples_2.11-3.0.0-SNAPSHOT.jar 4
18/10/31 12:02:30 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
18/10/31 12:02:30 INFO SparkKubernetesClientFactory: Auto-configuring K8S client using current context from users K8S config file
18/10/31 12:02:31 WARN WatchConnectionManager: Exec Failure
javax.net.ssl.SSLHandshakeException: sun.security.validator.ValidatorException: PKIX path building failed: sun.security.provider.certpath.SunCertPathBuilderException: unable to find valid certification path to requested target
	at sun.security.ssl.Alerts.getSSLException(Alerts.java:192)
	at sun.security.ssl.SSLSocketImpl.fatal(SSLSocketImpl.java:1949)
	at sun.security.ssl.Handshaker.fatalSE(Handshaker.java:302)
	at sun.security.ssl.Handshaker.fatalSE(Handshaker.java:296)
	at sun.security.ssl.ClientHandshaker.serverCertificate(ClientHandshaker.java:1509)
	at sun.security.ssl.ClientHandshaker.processMessage(ClientHandshaker.java:216)
	at sun.security.ssl.Handshaker.processLoop(Handshaker.java:979)
	at sun.security.ssl.Handshaker.process_record(Handshaker.java:914)
	at sun.security.ssl.SSLSocketImpl.readRecord(SSLSocketImpl.java:1062)
	at sun.security.ssl.SSLSocketImpl.performInitialHandshake(SSLSocketImpl.java:1375)
	at sun.security.ssl.SSLSocketImpl.startHandshake(SSLSocketImpl.java:1403)
	at sun.security.ssl.SSLSocketImpl.startHandshake(SSLSocketImpl.java:1387)
	at okhttp3.internal.connection.RealConnection.connectTls(RealConnection.java:281)
	at okhttp3.internal.connection.RealConnection.establishProtocol(RealConnection.java:251)
	at okhttp3.internal.connection.RealConnection.connect(RealConnection.java:151)
	at okhttp3.internal.connection.StreamAllocation.findConnection(StreamAllocation.java:195)
	at okhttp3.internal.connection.StreamAllocation.findHealthyConnection(StreamAllocation.java:121)
	at okhttp3.internal.connection.StreamAllocation.newStream(StreamAllocation.java:100)
	at okhttp3.internal.connection.ConnectInterceptor.intercept(ConnectInterceptor.java:42)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:67)
	at okhttp3.internal.cache.CacheInterceptor.intercept(CacheInterceptor.java:93)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:67)
	at okhttp3.internal.http.BridgeInterceptor.intercept(BridgeInterceptor.java:93)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RetryAndFollowUpInterceptor.intercept(RetryAndFollowUpInterceptor.java:120)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:67)
	at io.fabric8.kubernetes.client.utils.BackwardsCompatibilityInterceptor.intercept(BackwardsCompatibilityInterceptor.java:119)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:67)
	at io.fabric8.kubernetes.client.utils.ImpersonatorInterceptor.intercept(ImpersonatorInterceptor.java:66)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:67)
	at io.fabric8.kubernetes.client.utils.HttpClientUtils$2.intercept(HttpClientUtils.java:109)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:67)
	at okhttp3.RealCall.getResponseWithInterceptorChain(RealCall.java:185)
	at okhttp3.RealCall$AsyncCall.execute(RealCall.java:135)
	at okhttp3.internal.NamedRunnable.run(NamedRunnable.java:32)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
Caused by: sun.security.validator.ValidatorException: PKIX path building failed: sun.security.provider.certpath.SunCertPathBuilderException: unable to find valid certification path to requested target
	at sun.security.validator.PKIXValidator.doBuild(PKIXValidator.java:387)
	at sun.security.validator.PKIXValidator.engineValidate(PKIXValidator.java:292)
	at sun.security.validator.Validator.validate(Validator.java:260)
	at sun.security.ssl.X509TrustManagerImpl.validate(X509TrustManagerImpl.java:324)
	at sun.security.ssl.X509TrustManagerImpl.checkTrusted(X509TrustManagerImpl.java:229)
	at sun.security.ssl.X509TrustManagerImpl.checkServerTrusted(X509TrustManagerImpl.java:124)
	at sun.security.ssl.ClientHandshaker.serverCertificate(ClientHandshaker.java:1491)
	... 39 more
Caused by: sun.security.provider.certpath.SunCertPathBuilderException: unable to find valid certification path to requested target
	at sun.security.provider.certpath.SunCertPathBuilder.build(SunCertPathBuilder.java:141)
	at sun.security.provider.certpath.SunCertPathBuilder.engineBuild(SunCertPathBuilder.java:126)
	at java.security.cert.CertPathBuilder.build(CertPathBuilder.java:280)
	at sun.security.validator.PKIXValidator.doBuild(PKIXValidator.java:382)
	... 45 more
Exception in thread "kubernetes-dispatcher-0" Exception in thread "main" java.util.concurrent.RejectedExecutionException: Task java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask611a9c09 rejected from java.util.concurrent.ScheduledThreadPoolExecutor404819e4[Terminated, pool size = 0, active threads = 0, queued tasks = 0, completed tasks = 0]
	at java.util.concurrent.ThreadPoolExecutor$AbortPolicy.rejectedExecution(ThreadPoolExecutor.java:2047)
	at java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:823)
	at java.util.concurrent.ScheduledThreadPoolExecutor.delayedExecute(ScheduledThreadPoolExecutor.java:326)
	at java.util.concurrent.ScheduledThreadPoolExecutor.schedule(ScheduledThreadPoolExecutor.java:533)
	at java.util.concurrent.ScheduledThreadPoolExecutor.submit(ScheduledThreadPoolExecutor.java:632)
	at java.util.concurrent.Executors$DelegatedExecutorService.submit(Executors.java:678)
	at io.fabric8.kubernetes.client.dsl.internal.WatchConnectionManager.scheduleReconnect(WatchConnectionManager.java:300)
	at io.fabric8.kubernetes.client.dsl.internal.WatchConnectionManager.access$800(WatchConnectionManager.java:48)
	at io.fabric8.kubernetes.client.dsl.internal.WatchConnectionManager$2.onFailure(WatchConnectionManager.java:213)
	at okhttp3.internal.ws.RealWebSocket.failWebSocket(RealWebSocket.java:543)
	at okhttp3.internal.ws.RealWebSocket$2.onFailure(RealWebSocket.java:208)
	at okhttp3.RealCall$AsyncCall.execute(RealCall.java:148)
	at okhttp3.internal.NamedRunnable.run(NamedRunnable.java:32)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
io.fabric8.kubernetes.client.KubernetesClientException: Failed to start websocket
	at io.fabric8.kubernetes.client.dsl.internal.WatchConnectionManager$2.onFailure(WatchConnectionManager.java:204)
	at okhttp3.internal.ws.RealWebSocket.failWebSocket(RealWebSocket.java:543)
	at okhttp3.internal.ws.RealWebSocket$2.onFailure(RealWebSocket.java:208)
	at okhttp3.RealCall$AsyncCall.execute(RealCall.java:148)
	at okhttp3.internal.NamedRunnable.run(NamedRunnable.java:32)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
Caused by: javax.net.ssl.SSLHandshakeException: sun.security.validator.ValidatorException: PKIX path building failed: sun.security.provider.certpath.SunCertPathBuilderException: unable to find valid certification path to requested target
	at sun.security.ssl.Alerts.getSSLException(Alerts.java:192)
	at sun.security.ssl.SSLSocketImpl.fatal(SSLSocketImpl.java:1949)
	at sun.security.ssl.Handshaker.fatalSE(Handshaker.java:302)
	at sun.security.ssl.Handshaker.fatalSE(Handshaker.java:296)
	at sun.security.ssl.ClientHandshaker.serverCertificate(ClientHandshaker.java:1509)
	at sun.security.ssl.ClientHandshaker.processMessage(ClientHandshaker.java:216)
	at sun.security.ssl.Handshaker.processLoop(Handshaker.java:979)
	at sun.security.ssl.Handshaker.process_record(Handshaker.java:914)
	at sun.security.ssl.SSLSocketImpl.readRecord(SSLSocketImpl.java:1062)
	at sun.security.ssl.SSLSocketImpl.performInitialHandshake(SSLSocketImpl.java:1375)
	at sun.security.ssl.SSLSocketImpl.startHandshake(SSLSocketImpl.java:1403)
	at sun.security.ssl.SSLSocketImpl.startHandshake(SSLSocketImpl.java:1387)
	at okhttp3.internal.connection.RealConnection.connectTls(RealConnection.java:281)
	at okhttp3.internal.connection.RealConnection.establishProtocol(RealConnection.java:251)
	at okhttp3.internal.connection.RealConnection.connect(RealConnection.java:151)
	at okhttp3.internal.connection.StreamAllocation.findConnection(StreamAllocation.java:195)
	at okhttp3.internal.connection.StreamAllocation.findHealthyConnection(StreamAllocation.java:121)
	at okhttp3.internal.connection.StreamAllocation.newStream(StreamAllocation.java:100)
	at okhttp3.internal.connection.ConnectInterceptor.intercept(ConnectInterceptor.java:42)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:67)
	at okhttp3.internal.cache.CacheInterceptor.intercept(CacheInterceptor.java:93)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:67)
	at okhttp3.internal.http.BridgeInterceptor.intercept(BridgeInterceptor.java:93)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RetryAndFollowUpInterceptor.intercept(RetryAndFollowUpInterceptor.java:120)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:67)
	at io.fabric8.kubernetes.client.utils.BackwardsCompatibilityInterceptor.intercept(BackwardsCompatibilityInterceptor.java:119)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:67)
	at io.fabric8.kubernetes.client.utils.ImpersonatorInterceptor.intercept(ImpersonatorInterceptor.java:66)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:67)
	at io.fabric8.kubernetes.client.utils.HttpClientUtils$2.intercept(HttpClientUtils.java:109)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:92)
	at okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:67)
	at okhttp3.RealCall.getResponseWithInterceptorChain(RealCall.java:185)
	at okhttp3.RealCall$AsyncCall.execute(RealCall.java:135)
	... 4 more
Caused by: sun.security.validator.ValidatorException: PKIX path building failed: sun.security.provider.certpath.SunCertPathBuilderException: unable to find valid certification path to requested target
	at sun.security.validator.PKIXValidator.doBuild(PKIXValidator.java:387)
	at sun.security.validator.PKIXValidator.engineValidate(PKIXValidator.java:292)
	at sun.security.validator.Validator.validate(Validator.java:260)
	at sun.security.ssl.X509TrustManagerImpl.validate(X509TrustManagerImpl.java:324)
	at sun.security.ssl.X509TrustManagerImpl.checkTrusted(X509TrustManagerImpl.java:229)
	at sun.security.ssl.X509TrustManagerImpl.checkServerTrusted(X509TrustManagerImpl.java:124)
	at sun.security.ssl.ClientHandshaker.serverCertificate(ClientHandshaker.java:1491)
	... 39 more
Caused by: sun.security.provider.certpath.SunCertPathBuilderException: unable to find valid certification path to requested target
	at sun.security.provider.certpath.SunCertPathBuilder.build(SunCertPathBuilder.java:141)
	at sun.security.provider.certpath.SunCertPathBuilder.engineBuild(SunCertPathBuilder.java:126)
	at java.security.cert.CertPathBuilder.build(CertPathBuilder.java:280)
	at sun.security.validator.PKIXValidator.doBuild(PKIXValidator.java:382)
	... 45 more
18/10/31 12:02:31 INFO ShutdownHookManager: Shutdown hook called
18/10/31 12:02:31 INFO ShutdownHookManager: Deleting directory /private/var/folders/6b/y1010qp107j9w2dhhy8csvz0000xq3/T/spark-5e649891-8a0f-4f17-bf3a-33b34082eba8
```

Suggested reviews: mccheah liyinan926 - this is the follow up fix to the bug discovered while working on SPARK-25809 (PR #22805)

Closes #22904 from rvesse/SPARK-25887.

Authored-by: Rob Vesse <rvesse@dotnetrdf.org>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-01-22 10:25:21 -08:00
Shixiong Zhu 66450bbc1b
[SPARK-26665][CORE] Fix a bug that BlockTransferService.fetchBlockSync may hang forever
## What changes were proposed in this pull request?

`ByteBuffer.allocate` may throw `OutOfMemoryError` when the block is large but no enough memory is available. However, when this happens, right now BlockTransferService.fetchBlockSync will just hang forever as its `BlockFetchingListener. onBlockFetchSuccess` doesn't complete `Promise`.

This PR catches `Throwable` and uses the error to complete `Promise`.

## How was this patch tested?

Added a unit test. Since I cannot make `ByteBuffer.allocate` throw `OutOfMemoryError`, I passed a negative size to make `ByteBuffer.allocate` fail. Although the error type is different, it should trigger the same code path.

Closes #23590 from zsxwing/SPARK-26665.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2019-01-22 09:00:52 -08:00
Maxim Gekk 64ce1c9f93 [SPARK-26657][SQL] Use Proleptic Gregorian calendar in DayWeek and in WeekOfYear
## What changes were proposed in this pull request?

The expressions `DayWeek`, `DayOfWeek`, `WeekDay` and `WeekOfYear` are changed to use Proleptic Gregorian calendar instead of the hybrid one (Julian+Gregorian). This was achieved by using Java 8 API for date/timestamp manipulation, in particular the `LocalDate` class.

Week of year calculation is performed according to ISO-8601. The first week of a week-based-year is the first Monday-based week of the standard ISO year that has at least 4 days in the new year (see https://docs.oracle.com/javase/8/docs/api/java/time/temporal/IsoFields.html).

## How was this patch tested?

The changes were tested by `DateExpressionsSuite` and `DateFunctionsSuite`.

Closes #23594 from MaxGekk/dayweek-gregorian.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
2019-01-22 17:33:29 +01:00
Huaxin Gao 06792afd4c [SPARK-16838][PYTHON] Add PMML export for ML KMeans in PySpark
## What changes were proposed in this pull request?

Add PMML export support for ML KMeans to PySpark.

## How was this patch tested?

Add tests in ml.tests.PersistenceTest.

Closes #23592 from huaxingao/spark-16838.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-22 09:34:59 -06:00
Nihar Sheth 055bf8ea1f [SPARK-24938][CORE] Prevent Netty from using onheap memory for headers without regard for configuration
## What changes were proposed in this pull request?

In MessageEncoder.java, the header would always be allocated on onheap memory regardless of whether netty was configured to use/prefer onheap or offheap. By default this made netty allocate 16mb of onheap memory for a tiny header message. It would be more practical to use preallocated buffers.

Using a memory monitor tool on a simple spark application, the following services currently allocate 16 mb of onheap memory:
netty-rpc-client
netty-blockTransfer-client
netty-external-shuffle-client

With this change, the memory monitor tool reports all three of these services as using 0 b of onheap memory. The offheap memory allocation does not increase, but more of the already-allocated space is used.

## How was this patch tested?

Manually tested change using spark-memory-tool https://github.com/squito/spark-memory

Closes #22114 from NiharS/nettybuffer.

Lead-authored-by: Nihar Sheth <niharrsheth@gmail.com>
Co-authored-by: Nihar Sheth <nsheth@cloudera.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-22 08:41:42 -06:00
Kazuaki Ishizaki 7bf0794651 [SPARK-26463][CORE] Use ConfigEntry for hardcoded configs for scheduler categories.
## What changes were proposed in this pull request?

The PR makes hardcoded `spark.dynamicAllocation`, `spark.scheduler`, `spark.rpc`, `spark.task`, `spark.speculation`, and `spark.cleaner` configs to use `ConfigEntry`.

## How was this patch tested?

Existing tests

Closes #23416 from kiszk/SPARK-26463.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-22 07:44:36 -06:00
Jatin Puri d2e86cb3cd [SPARK-26616][MLLIB] Expose document frequency in IDFModel
## What changes were proposed in this pull request?

This change exposes the `df` (document frequency) as a public val along with the number of documents (`m`) as part of the IDF model.

* The document frequency is returned as an `Array[Long]`
* If the minimum  document frequency is set, this is considered in the df calculation. If the count is less than minDocFreq, the df is 0 for such terms
* numDocs is not very required. But it can be useful, if we plan to provide a provision in future for user to give their own idf function, instead of using a default (log((1+m)/(1+df))). In such cases, the user can provide a function taking input of `m` and `df` and returning the idf value
* Pyspark changes

## How was this patch tested?

The existing test case was edited to also check for the document frequency values.

I  am not very good with python or pyspark. I have committed and run tests based on my understanding. Kindly let me know if I have missed anything

Reviewer request: mengxr  zjffdu yinxusen

Closes #23549 from purijatin/master.

Authored-by: Jatin Puri <purijatin@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-22 07:41:54 -06:00
Liang-Chi Hsieh f92d276653 [SPARK-25811][PYSPARK] Raise a proper error when unsafe cast is detected by PyArrow
## What changes were proposed in this pull request?

Since 0.11.0, PyArrow supports to raise an error for unsafe cast ([PR](https://github.com/apache/arrow/pull/2504)). We should use it to raise a proper error for pandas udf users when such cast is detected.

Added a SQL config `spark.sql.execution.pandas.arrowSafeTypeConversion` to disable Arrow safe type check.

## How was this patch tested?

Added test and manually test.

Closes #22807 from viirya/SPARK-25811.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-22 14:54:41 +08:00
Wenchen Fan 098a2c41fc [SPARK-26520][SQL] data source v2 API refactor (micro-batch read)
## What changes were proposed in this pull request?

Following https://github.com/apache/spark/pull/23086, this PR does the API refactor for micro-batch read, w.r.t. the [doc](https://docs.google.com/document/d/1uUmKCpWLdh9vHxP7AWJ9EgbwB_U6T3EJYNjhISGmiQg/edit?usp=sharing)

The major changes:
1. rename `XXXMicroBatchReadSupport` to `XXXMicroBatchReadStream`
2. implement `TableProvider`, `Table`, `ScanBuilder` and `Scan` for streaming sources
3. at the beginning of micro-batch streaming execution, convert `StreamingRelationV2` to `StreamingDataSourceV2Relation` directly, instead of `StreamingExecutionRelation`.

followup:
support operator pushdown for stream sources

## How was this patch tested?

existing tests

Closes #23430 from cloud-fan/micro-batch.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-21 14:29:12 -08:00
Hyukjin Kwon 75d84498a4 [SPARK-26676][PYTHON] Make HiveContextSQLTests.test_unbounded_frames test compatible with Python 2 and PyPy
## What changes were proposed in this pull request?

This particular test is being skipped at PyPy and Python 2.

```
Skipped tests in pyspark.sql.tests.test_context with pypy:
    test_unbounded_frames (pyspark.sql.tests.test_context.HiveContextSQLTests) ... skipped "Unittest < 3.3 doesn't support mocking"

Skipped tests in pyspark.sql.tests.test_context with python2.7:
    test_unbounded_frames (pyspark.sql.tests.test_context.HiveContextSQLTests) ... skipped "Unittest < 3.3 doesn't support mocking"
```

We don't have to use unittest 3.3 module to mock. And looks the test itself isn't compatible with Python 2.

This PR makes:
 - Manually monkey-patch `sys.maxsize` to get rid of unittest 3.3 condition
 - Use the built-in `reload` in Python 2, and `importlib.reload` in Python 3

## How was this patch tested?

Manually tested, and unit test is fixed.

Closes #23604 from HyukjinKwon/test-window.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-01-21 14:27:17 -08:00
Maxim Gekk 4c1cd809f8 [SPARK-26652][SQL] Remove fromJSON and fromString from Literal
## What changes were proposed in this pull request?

The `fromString` and `fromJSON` methods of the `Literal` object are removed because they are not used.

Closes #23596

Closes #23603 from MaxGekk/remove-literal-fromstring.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-22 02:24:12 +08:00
Shahid 9a30e23211 [SPARK-26351][MLLIB] Update doc and minor correction in the mllib evaluation metrics
## What changes were proposed in this pull request?
Currently, there are some minor inconsistencies in doc compared to the code. In this PR, I am correcting those inconsistencies.
1) Links related to the evaluation metrics in the docs are not working
2) Minor correction in the evaluation metrics formulas in docs.

## How was this patch tested?

NA

Closes #23589 from shahidki31/docCorrection.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-20 18:11:14 -06:00
Luca Canali 6c18d8d807 [SPARK-26642][K8S] Add --num-executors option to spark-submit for Spark on K8S.
## What changes were proposed in this pull request?

This PR proposes to extend the spark-submit option --num-executors to be applicable to Spark on K8S too. It is motivated by convenience, for example when migrating jobs written for YARN to run on K8S.

## How was this patch tested?

Manually tested on a K8S cluster.

Author: Luca Canali <luca.canali@cern.ch>

Closes #23573 from LucaCanali/addNumExecutorsToK8s.
2019-01-20 12:43:34 -08:00
Marco Gaido 6d9c54b62c [SPARK-26645][PYTHON] Support decimals with negative scale when parsing datatype
## What changes were proposed in this pull request?

When parsing datatypes from the json internal representation, PySpark doesn't support decimals with negative scales. Since they are allowed and can actually happen, PySpark should be able to successfully parse them.

## How was this patch tested?

added test

Closes #23575 from mgaido91/SPARK-26645.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-20 17:43:50 +08:00
liuxian ace2364296 [MINOR][TEST] Correct some unit test mistakes
## What changes were proposed in this pull request?

Correct some unit test mistakes.

## How was this patch tested?
N/A

Closes #23583 from 10110346/unused_symbol.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-19 08:54:55 -06:00
Kazuaki Ishizaki 64cc9e572e
[SPARK-26477][CORE] Use ConfigEntry for hardcoded configs for unsafe category
## What changes were proposed in this pull request?

The PR makes hardcoded `spark.unsafe` configs to use ConfigEntry and put them in the `config` package.

## How was this patch tested?

Existing UTs

Closes #23412 from kiszk/SPARK-26477.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-18 23:57:04 -08:00
Liang-Chi Hsieh 8503aa3007 [SPARK-26646][TEST][PYSPARK] Fix flaky test: pyspark.mllib.tests.test_streaming_algorithms StreamingLogisticRegressionWithSGDTests.test_training_and_prediction
## What changes were proposed in this pull request?

The test pyspark.mllib.tests.test_streaming_algorithms StreamingLogisticRegressionWithSGDTests.test_training_and_prediction looks sometimes flaky.

```
======================================================================
FAIL: test_training_and_prediction (pyspark.mllib.tests.test_streaming_algorithms.StreamingLogisticRegressionWithSGDTests)
Test that the model improves on toy data with no. of batches
----------------------------------------------------------------------
Traceback (most recent call last):
  File "/home/jenkins/workspace/SparkPullRequestBuilder/python/pyspark/mllib/tests/test_streaming_algorithms.py", line 367, in test_training_and_prediction
    self._eventually(condition, timeout=60.0)
  File "/home/jenkins/workspace/SparkPullRequestBuilder/python/pyspark/mllib/tests/test_streaming_algorithms.py", line 69, in _eventually
    lastValue = condition()
  File "/home/jenkins/workspace/SparkPullRequestBuilder/python/pyspark/mllib/tests/test_streaming_algorithms.py", line 362, in condition
    self.assertGreater(errors[1] - errors[-1], 0.3)
AssertionError: -0.070000000000000062 not greater than 0.3

----------------------------------------------------------------------
Ran 13 tests in 198.327s

FAILED (failures=1, skipped=1)

Had test failures in pyspark.mllib.tests.test_streaming_algorithms with python3.4; see logs
```

The predict stream can possibly be consumed to the end before the input stream. When it happens, the model improvement is not high as expected and causes test failed. This patch tries to increase number of batches of streams. This won't increase test time because we have a timeout there.

## How was this patch tested?

Manually test.

Closes #23586 from viirya/SPARK-26646.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-18 23:53:11 +08:00
Maxim Gekk 34db5f5652 [SPARK-26618][SQL] Make typed Timestamp/Date literals consistent to casting
## What changes were proposed in this pull request?

In the PR, I propose to make creation of typed Literals `TIMESTAMP` and `DATE` consistent to the `Cast` expression. More precisely, reusing the `Cast` expression in the type constructors. In this way, it allows:
- To use the same calendar in parsing methods
- To support the same set of timestamp/date patterns

For example, creating timestamp literal:
```sql
SELECT TIMESTAMP '2019-01-14 20:54:00.000'
```
behaves similarly as casting the string literal:
```sql
SELECT CAST('2019-01-14 20:54:00.000' AS TIMESTAMP)
```

## How was this patch tested?

This was tested by `SQLQueryTestSuite` as well as `ExpressionParserSuite`.

Closes #23541 from MaxGekk/timestamp-date-constructors.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
2019-01-18 12:47:36 +01:00
Kris Mok e3418649dc [SPARK-26659][SQL] Fix duplicate cmd.nodeName in the explain output of DataWritingCommandExec
## What changes were proposed in this pull request?

`DataWritingCommandExec` generates `cmd.nodeName` twice in its explain output, e.g. when running this query `spark.sql("create table foo stored as parquet as select id, id % 10 as cat1, id % 20 as cat2 from range(10)")`,
```
Execute OptimizedCreateHiveTableAsSelectCommand OptimizedCreateHiveTableAsSelectCommand [Database:default, TableName: foo, InsertIntoHiveTable]
+- *(1) Project [id#2L, (id#2L % 10) AS cat1#0L, (id#2L % 20) AS cat2#1L]
   +- *(1) Range (0, 10, step=1, splits=8)
```
After the fix, it'll go back to normal:
```
Execute OptimizedCreateHiveTableAsSelectCommand [Database:default, TableName: foo, InsertIntoHiveTable]
+- *(1) Project [id#2L, (id#2L % 10) AS cat1#0L, (id#2L % 20) AS cat2#1L]
   +- *(1) Range (0, 10, step=1, splits=8)
```

This duplication is introduced when this specialized `DataWritingCommandExec` was created in place of `ExecutedCommandExec`.

The former is a `UnaryExecNode` whose `children` include the physical plan of the query, and the `cmd` is picked up via `TreeNode.stringArgs` into the argument string. The duplication comes from: `DataWritingCommandExec.nodeName` is `s"Execute ${cmd.nodeName}"` while the argument string is `cmd.simpleString()` which also includes `cmd.nodeName`.

The latter didn't have that problem because it's a `LeafExecNode` with no children, and it declares the `cmd` as being a part of the `innerChildren` which is excluded from the argument string.

## How was this patch tested?

Manual testing of running the example above in a local Spark Shell.
Also added a new test case in `ExplainSuite`.

Closes #23579 from rednaxelafx/fix-explain.

Authored-by: Kris Mok <kris.mok@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-17 22:43:39 -08:00
Sean Owen c2d0d700b5 [SPARK-26640][CORE][ML][SQL][STREAMING][PYSPARK] Code cleanup from lgtm.com analysis
## What changes were proposed in this pull request?

Misc code cleanup from lgtm.com analysis. See comments below for details.

## How was this patch tested?

Existing tests.

Closes #23571 from srowen/SPARK-26640.

Lead-authored-by: Sean Owen <sean.owen@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Co-authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-17 19:40:39 -06:00
Sean Owen 0b3abef195 [SPARK-26638][PYSPARK][ML] Pyspark vector classes always return error for unary negation
## What changes were proposed in this pull request?

Fix implementation of unary negation (`__neg__`) in Pyspark DenseVectors

## How was this patch tested?

Existing tests, plus new doctest

Closes #23570 from srowen/SPARK-26638.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-17 14:24:21 -06:00
Juliusz Sompolski ede35c88e0 [SPARK-26622][SQL] Revise SQL Metrics labels
## What changes were proposed in this pull request?

Try to make labels more obvious
"avg hash probe"	avg hash probe bucket iterations
"partition pruning time (ms)"	dynamic partition pruning time
"total number of files in the table"	file count
"number of files that would be returned by partition pruning alone"	file count after partition pruning
"total size of files in the table"	file size
"size of files that would be returned by partition pruning alone"	file size after partition pruning
"metadata time (ms)"	metadata time
"aggregate time"	time in aggregation build
"aggregate time"	time in aggregation build
"time to construct rdd bc"	time to build
"total time to remove rows"	time to remove
"total time to update rows"	time to update

Add proper metric type to some metrics:
"bytes of written output"	written output - createSizeMetric
"metadata time"	- createTimingMetric
"dataSize"	- createSizeMetric
"collectTime"	- createTimingMetric
"buildTime"	- createTimingMetric
"broadcastTIme"	- createTimingMetric

## How is this patch tested?

Existing tests.

Author: Stacy Kerkela <stacy.kerkeladatabricks.com>
Signed-off-by: Juliusz Sompolski <julekdatabricks.com>

Closes #23551 from juliuszsompolski/SPARK-26622.

Lead-authored-by: Juliusz Sompolski <julek@databricks.com>
Co-authored-by: Stacy Kerkela <stacy.kerkela@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-17 10:49:42 -08:00
liuxian 1b575ef5d1 [SPARK-26621][CORE] Use ConfigEntry for hardcoded configs for shuffle categories.
## What changes were proposed in this pull request?

The PR makes hardcoded `spark.shuffle` configs to use ConfigEntry and put them in the config package.

## How was this patch tested?
Existing unit tests

Closes #23550 from 10110346/ConfigEntry_shuffle.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-17 12:29:17 -06:00
Maxim Gekk 6f8c0e5255 [SPARK-26593][SQL] Use Proleptic Gregorian calendar in casting UTF8String to Date/TimestampType
## What changes were proposed in this pull request?

In the PR, I propose to use *java.time* classes in `stringToDate` and `stringToTimestamp`. This switches the methods from the hybrid calendar (Gregorian+Julian) to Proleptic Gregorian calendar. And it should make the casting consistent to other Spark classes that converts textual representation of dates/timestamps to `DateType`/`TimestampType`.

## How was this patch tested?

The changes were tested by existing suites - `HashExpressionsSuite`, `CastSuite` and `DateTimeUtilsSuite`.

Closes #23512 from MaxGekk/utf8string-timestamp-parsing.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
2019-01-17 17:53:00 +01:00
Gengliang Wang c0632cec04 [SPARK-23817][SQL] Create file source V2 framework and migrate ORC read path
## What changes were proposed in this pull request?
Create a framework for file source V2 based on data source V2 API.
As a good example for demonstrating the framework, this PR also migrate ORC source. This is because ORC file source supports both row scan and columnar scan, and the implementation is simpler comparing with Parquet.

Note: Currently only read path of V2 API is done, this framework and migration are only for the read path.
Supports the following scan:
- Scan ColumnarBatch
- Scan UnsafeRow
- Push down filters
- Push down required columns

Not supported( due to the limitation of data source V2 API):
- Stats metrics
- Catalog table
- Writes

## How was this patch tested?

Unit test

Closes #23383 from gengliangwang/latest_orcV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-17 23:33:29 +08:00
xiaodeshan 650b879de9 [SPARK-26457] Show hadoop configurations in HistoryServer environment tab
## What changes were proposed in this pull request?

I know that yarn provided all hadoop configurations. But I guess it may be fine that the historyserver unify all configuration in it. It will be convenient for us to debug some problems.

## How was this patch tested?

![image](https://user-images.githubusercontent.com/42019462/50808610-4d742900-133a-11e9-868c-2976e856ed9a.png)

Closes #23486 from deshanxiao/spark-26457.

Lead-authored-by: xiaodeshan <xiaodeshan@xiaomi.com>
Co-authored-by: deshanxiao <42019462+deshanxiao@users.noreply.github.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-17 05:51:43 -06:00
wright 4915cb3adf [MINOR][BUILD] ensure call to translate_component has correct number of arguments
## What changes were proposed in this pull request?

The call to `translate_component` only supplied 2 out of the 3 required arguments. I added a default empty list for the missing argument to avoid a run-time error.

I work for Semmle, and noticed the bug with our LGTM code analyzer:
0655f1624f/files/dev/create-release/releaseutils.py?sort=name&dir=ASC&mode=heatmap#x1434915b6576fb40:1

## How was this patch tested?

I checked that  `./dev/run-tests` pass OK.

Closes #23567 from ipwright/wrong-number-of-arguments-fix.

Authored-by: wright <wright@semmle.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-16 21:00:58 -06:00