Commit graph

23478 commits

Author SHA1 Message Date
cody koeninger 9a09e91a3e [SPARK-26177] Automated formatting for Scala code
## What changes were proposed in this pull request?

Add a maven plugin and wrapper script to use scalafmt to format files that differ from git master.

Intention is for contributors to be able to use this to automate fixing code style, not to include it in build pipeline yet.

If this PR is accepted, I'd make a different PR to update the code style section of https://spark.apache.org/contributing.html to mention the script

## How was this patch tested?

Manually tested by modifying a few files and running ./dev/scalafmt then checking that ./dev/scalastyle still passed.

Closes #23148 from koeninger/scalafmt.

Authored-by: cody koeninger <cody@koeninger.org>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-29 08:54:31 -06:00
zhengruifeng e3ea93ab6c [MINOR][ML] add missing params to Instr
## What changes were proposed in this pull request?
add following param to instr:
GBTC: validationTol
GBTR: validationTol, validationIndicatorCol
colnames in LiR, LinearSVC, etc

## How was this patch tested?
existing tests

Closes #23122 from zhengruifeng/instr_append_missing_params.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-29 08:53:12 -06:00
Sean Owen 06a87711b8 [SPARK-26024][FOLLOWUP][MINOR] Follow-up to remove extra blank lines in R function descriptions
## What changes were proposed in this pull request?

Follow-up to remove extra blank lines in R function descriptions

## How was this patch tested?

N/A

Closes #23167 from srowen/SPARK-26024.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-29 08:48:12 -06:00
Takuya UESHIN b9b68a6dc7 [SPARK-26211][SQL] Fix InSet for binary, and struct and array with null.
## What changes were proposed in this pull request?

Currently `InSet` doesn't work properly for binary type, or struct and array type with null value in the set.
Because, as for binary type, the `HashSet` doesn't work properly for `Array[Byte]`, and as for struct and array type with null value in the set, the `ordering` will throw a `NPE`.

## How was this patch tested?

Added a few tests.

Closes #23176 from ueshin/issues/SPARK-26211/inset.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-29 22:37:02 +08:00
Maxim Gekk 7a83d71403 [SPARK-26163][SQL] Parsing decimals from JSON using locale
## What changes were proposed in this pull request?

In the PR, I propose using of the locale option to parse (and infer) decimals from JSON input. After the changes, `JacksonParser` converts input string to `BigDecimal` and to Spark's Decimal by using `java.text.DecimalFormat`. New behaviour can be switched off via SQL config `spark.sql.legacy.decimalParsing.enabled`.

## How was this patch tested?

Added 2 tests to `JsonExpressionsSuite` for the `en-US`, `ko-KR`, `ru-RU`, `de-DE` locales:
- Inferring decimal type using locale from JSON field values
- Converting JSON field values to specified decimal type using the locales.

Closes #23132 from MaxGekk/json-decimal-parsing-locale.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-29 22:15:12 +08:00
Liang-Chi Hsieh 8bfea86b1c
[SPARK-26133][ML] Remove deprecated OneHotEncoder and rename OneHotEncoderEstimator to OneHotEncoder
## What changes were proposed in this pull request?

We have deprecated `OneHotEncoder` at Spark 2.3.0 and introduced `OneHotEncoderEstimator`. At 3.0.0, we remove deprecated `OneHotEncoder` and rename `OneHotEncoderEstimator` to `OneHotEncoder`.

TODO: According to ML migration guide, we need to keep `OneHotEncoderEstimator` as an alias after renaming. This is not done at this patch in order to facilitate review.

## How was this patch tested?

Existing tests.

Closes #23100 from viirya/remove_one_hot_encoder.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2018-11-29 01:54:06 +00:00
Wenchen Fan fa0d4bf699 [SPARK-25829][SQL] remove duplicated map keys with last wins policy
## What changes were proposed in this pull request?

Currently duplicated map keys are not handled consistently. For example, map look up respects the duplicated key appears first, `Dataset.collect` only keeps the duplicated key appears last, `MapKeys` returns duplicated keys, etc.

This PR proposes to remove duplicated map keys with last wins policy, to follow Java/Scala and Presto. It only applies to built-in functions, as users can create map with duplicated map keys via private APIs anyway.

updated functions: `CreateMap`, `MapFromArrays`, `MapFromEntries`, `StringToMap`, `MapConcat`, `TransformKeys`.

For other places:
1. data source v1 doesn't have this problem, as users need to provide a java/scala map, which can't have duplicated keys.
2. data source v2 may have this problem. I've added a note to `ArrayBasedMapData` to ask the caller to take care of duplicated keys. In the future we should enforce it in the stable data APIs for data source v2.
3. UDF doesn't have this problem, as users need to provide a java/scala map. Same as data source v1.
4. file format. I checked all of them and only parquet does not enforce it. For backward compatibility reasons I change nothing but leave a note saying that the behavior will be undefined if users write map with duplicated keys to parquet files. Maybe we can add a config and fail by default if parquet files have map with duplicated keys. This can be done in followup.

## How was this patch tested?

updated tests and new tests

Closes #23124 from cloud-fan/map.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-28 23:42:13 +08:00
zhengruifeng 9fde3deab8 [SPARK-25989][ML] OneVsRestModel handle empty outputCols incorrectly
## What changes were proposed in this pull request?
ignore empty output columns

## How was this patch tested?
added tests

Closes #22991 from zhengruifeng/ovrm_empty_outcol.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-28 07:33:34 -08:00
Brandon Krieger 87bd9c75df [SPARK-25998][CORE] Change TorrentBroadcast to hold weak reference of broadcast object
## What changes were proposed in this pull request?

This PR changes the broadcast object in TorrentBroadcast from a strong reference to a weak reference. This allows it to be garbage collected even if the Dataset is held in memory. This is ok, because the broadcast object can always be re-read.

## How was this patch tested?

Tested in Spark shell by taking a heap dump, full repro steps listed in https://issues.apache.org/jira/browse/SPARK-25998.

Closes #22995 from bkrieger/bk/torrent-broadcast-weak.

Authored-by: Brandon Krieger <bkrieger@palantir.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-28 07:22:48 -08:00
Mark Pavey ce61bac1d8 [SPARK-26137][CORE] Use Java system property "file.separator" inste…
… of hard coded "/" in DependencyUtils

## What changes were proposed in this pull request?

Use Java system property "file.separator" instead of hard coded "/" in DependencyUtils.

## How was this patch tested?

Manual test:
Submit Spark application via REST API that reads data from Elasticsearch using spark-elasticsearch library.

Without fix application fails with error:
18/11/22 10:36:20 ERROR Version: Multiple ES-Hadoop versions detected in the classpath; please use only one
jar:file:/C:/<...>/spark-2.4.0-bin-hadoop2.6/work/driver-20181122103610-0001/myApp-assembly-1.0.jar
jar:file:/C:/<...>/myApp-assembly-1.0.jar

18/11/22 10:36:20 ERROR Main: Application [MyApp] failed:
java.lang.Error: Multiple ES-Hadoop versions detected in the classpath; please use only one
jar:file:/C:/<...>/spark-2.4.0-bin-hadoop2.6/work/driver-20181122103610-0001/myApp-assembly-1.0.jar
jar:file:/C:/<...>/myApp-assembly-1.0.jar

	at org.elasticsearch.hadoop.util.Version.<clinit>(Version.java:73)
	at org.elasticsearch.hadoop.rest.RestService.findPartitions(RestService.java:214)
	at org.elasticsearch.spark.rdd.AbstractEsRDD.esPartitions$lzycompute(AbstractEsRDD.scala:73)
	at org.elasticsearch.spark.rdd.AbstractEsRDD.esPartitions(AbstractEsRDD.scala:72)
	at org.elasticsearch.spark.rdd.AbstractEsRDD.getPartitions(AbstractEsRDD.scala:44)
	at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:253)
	at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:251)
	at scala.Option.getOrElse(Option.scala:121)
	at org.apache.spark.rdd.RDD.partitions(RDD.scala:251)
	at org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:49)
	at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:253)
	at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:251)
	at scala.Option.getOrElse(Option.scala:121)
	at org.apache.spark.rdd.RDD.partitions(RDD.scala:251)
	at org.apache.spark.SparkContext.runJob(SparkContext.scala:2126)
	at org.apache.spark.rdd.RDD$$anonfun$collect$1.apply(RDD.scala:945)
	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
	at org.apache.spark.rdd.RDD.withScope(RDD.scala:363)
	at org.apache.spark.rdd.RDD.collect(RDD.scala:944)
	...
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.apache.spark.deploy.worker.DriverWrapper$.main(DriverWrapper.scala:65)
	at org.apache.spark.deploy.worker.DriverWrapper.main(DriverWrapper.scala)

With fix application runs successfully.

Closes #23102 from markpavey/JIRA_SPARK-26137_DependencyUtilsFileSeparatorFix.

Authored-by: Mark Pavey <markpavey@exabre.co.uk>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-28 07:19:47 -08:00
Wenchen Fan affe80958d [SPARK-26147][SQL] only pull out unevaluable python udf from join condition
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/22326 made a mistake that, not all python UDFs are unevaluable in join condition. Only python UDFs that refer to attributes from both join side are unevaluable.

This PR fixes this mistake.

## How was this patch tested?

a new test

Closes #23153 from cloud-fan/join.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-28 20:38:42 +08:00
Sergey Zhemzhitsky 438f8fd675 [SPARK-26114][CORE] ExternalSorter's readingIterator field leak
## What changes were proposed in this pull request?

This pull request fixes [SPARK-26114](https://issues.apache.org/jira/browse/SPARK-26114) issue that occurs when trying to reduce the number of partitions by means of coalesce without shuffling after shuffle-based transformations.

The leak occurs because of not cleaning up `ExternalSorter`'s `readingIterator` field as it's done for its `map` and `buffer` fields.
Additionally there are changes to the `CompletionIterator` to prevent capturing its `sub`-iterator and holding it even after the completion iterator completes. It is necessary because in some cases, e.g. in case of standard scala's `flatMap` iterator (which is used is `CoalescedRDD`'s `compute` method) the next value of the main iterator is assigned to `flatMap`'s `cur` field only after it is available.
For DAGs where ShuffledRDD is a parent of CoalescedRDD it means that the data should be fetched from the map-side of the shuffle, but the process of fetching this data consumes quite a lot of memory in addition to the memory already consumed by the iterator held by `flatMap`'s `cur` field (until it is reassigned).

For the following data
```scala
import org.apache.hadoop.io._
import org.apache.hadoop.io.compress._
import org.apache.commons.lang._
import org.apache.spark._

// generate 100M records of sample data
sc.makeRDD(1 to 1000, 1000)
  .flatMap(item => (1 to 100000)
    .map(i => new Text(RandomStringUtils.randomAlphanumeric(3).toLowerCase) -> new Text(RandomStringUtils.randomAlphanumeric(1024))))
  .saveAsSequenceFile("/tmp/random-strings", Some(classOf[GzipCodec]))
```

and the following job
```scala
import org.apache.hadoop.io._
import org.apache.spark._
import org.apache.spark.storage._

val rdd = sc.sequenceFile("/tmp/random-strings", classOf[Text], classOf[Text])
rdd
  .map(item => item._1.toString -> item._2.toString)
  .repartitionAndSortWithinPartitions(new HashPartitioner(1000))
  .coalesce(10,false)
  .count
```

... executed like the following
```bash
spark-shell \
  --num-executors=5 \
  --executor-cores=2 \
  --master=yarn \
  --deploy-mode=client \
  --conf spark.executor.memoryOverhead=512 \
  --conf spark.executor.memory=1g \
  --conf spark.dynamicAllocation.enabled=false \
  --conf spark.executor.extraJavaOptions='-XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/tmp -Dio.netty.noUnsafe=true'
```

... executors are always failing with OutOfMemoryErrors.

The main issue is multiple leaks of ExternalSorter references.
For example, in case of 2 tasks per executor it is expected to be 2 simultaneous instances of ExternalSorter per executor but heap dump generated on OutOfMemoryError shows that there are more ones.

![run1-noparams-dominator-tree-externalsorter](https://user-images.githubusercontent.com/1523889/48703665-782ce580-ec05-11e8-95a9-d6c94e8285ab.png)

P.S. This PR does not cover cases with CoGroupedRDDs which use ExternalAppendOnlyMap internally, which itself can lead to OutOfMemoryErrors in many places.

## How was this patch tested?

- Existing unit tests
- New unit tests
- Job executions on the live environment

Here is the screenshot before applying this patch
![run3-noparams-failure-ui-5x2-repartition-and-sort](https://user-images.githubusercontent.com/1523889/48700395-f769eb80-ebfc-11e8-831b-e94c757d416c.png)

Here is the screenshot after applying this patch
![run3-noparams-success-ui-5x2-repartition-and-sort](https://user-images.githubusercontent.com/1523889/48700610-7a8b4180-ebfd-11e8-9761-baaf38a58e66.png)
And in case of reducing the number of executors even more the job is still stable
![run3-noparams-success-ui-2x2-repartition-and-sort](https://user-images.githubusercontent.com/1523889/48700619-82e37c80-ebfd-11e8-98ed-a38e1f1f1fd9.png)

Closes #23083 from szhem/SPARK-26114-externalsorter-leak.

Authored-by: Sergey Zhemzhitsky <szhemzhitski@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-28 20:22:24 +08:00
Yuanjian Li 93112e6930 [SPARK-26142][SQL] Implement shuffle read metrics in SQL
## What changes were proposed in this pull request?

Implement `SQLShuffleMetricsReporter` on the sql side as the customized ShuffleMetricsReporter, which extended the `TempShuffleReadMetrics` and update SQLMetrics, in this way shuffle metrics can be reported in the SQL UI.

## How was this patch tested?

Add UT in SQLMetricsSuite.
Manual test locally, before:
![image](https://user-images.githubusercontent.com/4833765/48960517-30f97880-efa8-11e8-982c-92d05938fd1d.png)
after:
![image](https://user-images.githubusercontent.com/4833765/48960587-b54bfb80-efa8-11e8-8e95-7a3c8c74cc5c.png)

Closes #23128 from xuanyuanking/SPARK-26142.

Lead-authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Co-authored-by: liyuanjian <liyuanjian@baidu.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-28 20:18:13 +08:00
Wenchen Fan 09a91d98bd [SPARK-26021][SQL][FOLLOWUP] add test for special floating point values
## What changes were proposed in this pull request?

a followup of https://github.com/apache/spark/pull/23043 . Add a test to show the minor behavior change introduced by #23043 , and add migration guide.

## How was this patch tested?

a new test

Closes #23141 from cloud-fan/follow.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-28 16:21:42 +08:00
Juliusz Sompolski 8c6871828e [SPARK-26159] Codegen for LocalTableScanExec and RDDScanExec
## What changes were proposed in this pull request?

Implement codegen for `LocalTableScanExec` and `ExistingRDDExec`. Refactor to share code between `LocalTableScanExec`, `ExistingRDDExec`, `InputAdapter` and `RowDataSourceScanExec`.

The difference in `doProduce` between these four was that `ExistingRDDExec` and `RowDataSourceScanExec` triggered adding an `UnsafeProjection`, while `InputAdapter` and `LocalTableScanExec` did not.

In the new trait `InputRDDCodegen` I added a flag `createUnsafeProjection` which the operators set accordingly.

Note: `LocalTableScanExec` explicitly creates its input as `UnsafeRows`, so it was obvious why it doesn't need an `UnsafeProjection`. But if an `InputAdapter` may take input that is `InternalRows` but not `UnsafeRows`, then I think it doesn't need an unsafe projection just because any other operator that is its parent would do that. That assumes that that any parent operator would always result in some `UnsafeProjection` being eventually added, and hence the output of the `WholeStageCodegen` unit would be `UnsafeRows`. If these assumptions hold, I think `createUnsafeProjection` could be set to `(parent == null)`.

Note: Do not codegen `LocalTableScanExec` when it's the only operator. `LocalTableScanExec` has optimized driver-only `executeCollect` and `executeTake` code paths that are used to return `Command` results without starting Spark Jobs. They can no longer be used if the `LocalTableScanExec` gets optimized.

## How was this patch tested?

Covered and used in existing tests.

Closes #23127 from juliuszsompolski/SPARK-26159.

Authored-by: Juliusz Sompolski <julek@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-28 13:37:11 +08:00
Marcelo Vanzin 2d89d109e1 [SPARK-26025][K8S] Speed up docker image build on dev repo.
The "build context" for a docker image - basically the whole contents of the
current directory where "docker" is invoked - can be huge in a dev build,
easily breaking a couple of gigs.

Doing that copy 3 times during the build of docker images severely slows
down the process.

This patch creates a smaller build context - basically mimicking what the
make-distribution.sh script does, so that when building the docker images,
only the necessary bits are in the current directory. For PySpark and R that
is optimized further, since those images are built based on the previously
built Spark main image.

In my current local clone, the dir size is about 2G, but with this script
the "context" sent to docker is about 250M for the main image, 1M for the
pyspark image and 8M for the R image. That speeds up the image builds
considerably.

I also snuck in a fix to the k8s integration test dependencies in the sbt
build, so that the examples are properly built (without having to do it
manually).

Closes #23019 from vanzin/SPARK-26025.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-27 09:09:16 -08:00
caoxuewen 65244b1d79 [SPARK-23356][SQL][TEST] add new test cases for a + 1,a + b and Rand in SetOperationSuite
## What changes were proposed in this pull request?

The purpose of this PR is supplement new test cases for a + 1,a + b and Rand in SetOperationSuite.
It comes from the comment of closed PR:#20541, thanks.

## How was this patch tested?

add new test cases

Closes #23138 from heary-cao/UnionPushTestCases.

Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-27 20:10:34 +08:00
Reynold Xin 6a064ba8f2 [SPARK-26141] Enable custom metrics implementation in shuffle write
## What changes were proposed in this pull request?
This is the write side counterpart to https://github.com/apache/spark/pull/23105

## How was this patch tested?
No behavior change expected, as it is a straightforward refactoring. Updated all existing test cases.

Closes #23106 from rxin/SPARK-26141.

Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: Reynold Xin <rxin@databricks.com>
2018-11-26 22:35:52 -08:00
gatorsmile 85383d29ed
[SPARK-25860][SPARK-26107][FOLLOW-UP] Rule ReplaceNullWithFalseInPredicate
## What changes were proposed in this pull request?

Based on https://github.com/apache/spark/pull/22857 and https://github.com/apache/spark/pull/23079, this PR did a few updates

- Limit the data types of NULL to Boolean.
- Limit the input data type of replaceNullWithFalse to Boolean; throw an exception in the testing mode.
- Create a new file for the rule ReplaceNullWithFalseInPredicate
- Update the description of this rule.

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

Closes #23139 from gatorsmile/followupSpark-25860.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2018-11-27 04:51:32 +00:00
Liang-Chi Hsieh 1c487f7d14 [SPARK-24762][SQL][FOLLOWUP] Enable Option of Product encoders
## What changes were proposed in this pull request?

This is follow-up of #21732. This patch inlines `isOptionType` method.

## How was this patch tested?

Existing tests.

Closes #23143 from viirya/SPARK-24762-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-27 12:13:48 +08:00
Reynold Xin c995e0737d [SPARK-26140] followup: rename ShuffleMetricsReporter
## What changes were proposed in this pull request?
In https://github.com/apache/spark/pull/23105, due to working on two parallel PRs at once, I made the mistake of committing the copy of the PR that used the name ShuffleMetricsReporter for the interface, rather than the appropriate one ShuffleReadMetricsReporter. This patch fixes that.

## How was this patch tested?
This should be fine as long as compilation passes.

Closes #23147 from rxin/ShuffleReadMetricsReporter.

Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-11-26 17:01:56 -08:00
Marcelo Vanzin 9deaa726ef [INFRA] Close stale PR.
Closes #23107
2018-11-26 15:33:21 -08:00
Marcelo Vanzin 6f1a1c1248 [SPARK-25451][HOTFIX] Call stage.attemptNumber instead of attemptId.
Closes #23149 from vanzin/SPARK-25451.hotfix.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-26 14:37:41 -08:00
Shahid fbf62b7100 [SPARK-25451][SPARK-26100][CORE] Aggregated metrics table doesn't show the right number of the total tasks
Total tasks in the aggregated table and the tasks table are not matching some times in the WEBUI.
We need to force update the executor summary of the particular executorId, when ever last task of that executor has reached. Currently it force update based on last task on the stage end. So, for some particular executorId task might miss at the stage end.

Tests to reproduce:
```
bin/spark-shell --master yarn --conf spark.executor.instances=3
sc.parallelize(1 to 10000, 10).map{ x => throw new RuntimeException("Bad executor")}.collect()
```
Before patch:
![screenshot from 2018-11-15 02-24-05](https://user-images.githubusercontent.com/23054875/48511776-b0d36480-e87d-11e8-89a8-ab97216e2c21.png)

After patch:
![screenshot from 2018-11-15 02-32-38](https://user-images.githubusercontent.com/23054875/48512141-c39a6900-e87e-11e8-8535-903e1d11d13e.png)

Closes #23038 from shahidki31/SPARK-25451.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-26 13:13:45 -08:00
pgandhi 76ef02e499 [SPARK-21809] Change Stage Page to use datatables to support sorting columns and searching
Support column sort, pagination and search for Stage Page using jQuery DataTable and REST API. Before this commit, the Stage page generated a hard-coded HTML table that could not support search. Supporting search and sort (over all applications rather than the 20 entries in the current page) in any case will greatly improve the user experience.
Created the stagespage-template.html for displaying application information in datables. Added REST api endpoint and javascript code to fetch data from the endpoint and display it on the data table.
Because of the above change, certain functionalities in the page had to be modified to support the addition of datatables. For example, the toggle checkbox 'Select All' previously would add the checked fields as columns in the Task table and as rows in the Summary Metrics table, but after the change, only columns are added in the Task Table as it got tricky to add rows dynamically in the datatables.

## How was this patch tested?
I have attached the screenshots of the Stage Page UI before and after the fix.
**Before:**

<img width="1419" alt="30564304-35991e1c-9c8a-11e7-850f-2ac7a347f600" src="https://user-images.githubusercontent.com/22228190/42137915-52054558-7d3a-11e8-8c85-433b2c94161d.png">

<img width="1435" alt="31360592-cbaa2bae-ad14-11e7-941d-95b4c7d14970" src="https://user-images.githubusercontent.com/22228190/42137928-79df500a-7d3a-11e8-9068-5630afe46ff3.png">

**After:**

<img width="1432" alt="31360591-c5650ee4-ad14-11e7-9665-5a08d8f21830" src="https://user-images.githubusercontent.com/22228190/42137936-a3fb9f42-7d3a-11e8-8502-22b3897cbf64.png">

<img width="1388" alt="31360604-d266b6b0-ad14-11e7-94b5-dcc4bb5443f4" src="https://user-images.githubusercontent.com/22228190/42137970-0fabc58c-7d3b-11e8-95ad-383b1bd1f106.png">

Closes #21688 from pgandhi999/SPARK-21809-2.3.

Authored-by: pgandhi <pgandhi@oath.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2018-11-26 14:08:32 -06:00
Nihar Sheth 3df307aa51 [SPARK-25960][K8S] Support subpath mounting with Kubernetes
## What changes were proposed in this pull request?

This PR adds configurations to use subpaths with Spark on k8s. Subpaths (https://kubernetes.io/docs/concepts/storage/volumes/#using-subpath) allow the user to specify a path within a volume to use instead of the volume's root.

## How was this patch tested?

Added unit tests. Ran SparkPi on a cluster with event logging pointed at a subpath-mount and verified the driver host created and used the subpath.

Closes #23026 from NiharS/k8s_subpath.

Authored-by: Nihar Sheth <niharrsheth@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-26 11:06:02 -08:00
Anastasios Zouzias 2512a1d429 [SPARK-26121][STRUCTURED STREAMING] Allow users to define prefix of Kafka's consumer group (group.id)
## What changes were proposed in this pull request?

Allow the Spark Structured Streaming user to specify the prefix of the consumer group (group.id), compared to force consumer group ids of the form `spark-kafka-source-*`

## How was this patch tested?

Unit tests provided by Spark (backwards compatible change, i.e., user can optionally use the functionality)

`mvn test -pl external/kafka-0-10`

Closes #23103 from zouzias/SPARK-26121.

Authored-by: Anastasios Zouzias <anastasios@sqooba.io>
Signed-off-by: cody koeninger <cody@koeninger.org>
2018-11-26 11:10:38 -06:00
zhengruifeng 1bb60ab839 [SPARK-26153][ML] GBT & RandomForest avoid unnecessary first job to compute numFeatures
## What changes were proposed in this pull request?
use base models' `numFeature` instead of `first` job

## How was this patch tested?
existing tests

Closes #23123 from zhengruifeng/avoid_first_job.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-26 05:57:33 -06:00
gatorsmile 6bb60b30fd [SPARK-26168][SQL] Update the code comments in Expression and Aggregate
## What changes were proposed in this pull request?
This PR is to improve the code comments to document some common traits and traps about the expression.

## How was this patch tested?
N/A

Closes #23135 from gatorsmile/addcomments.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-26 15:51:28 +08:00
gatorsmile 6ab8485da2 [SPARK-26169] Create DataFrameSetOperationsSuite
## What changes were proposed in this pull request?

Create a new suite DataFrameSetOperationsSuite for the test cases of DataFrame/Dataset's set operations.

Also, add test cases of NULL handling for Array Except and Array Intersect.

## How was this patch tested?
N/A

Closes #23137 from gatorsmile/setOpsTest.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-26 15:47:04 +08:00
Liang-Chi Hsieh 6339c8c2c6 [SPARK-24762][SQL] Enable Option of Product encoders
## What changes were proposed in this pull request?

SparkSQL doesn't support to encode `Option[Product]` as a top-level row now, because in SparkSQL entire top-level row can't be null.

However for use cases like Aggregator, it is reasonable to use `Option[Product]` as buffer and output column types. Due to above limitation, we don't do it for now.

This patch proposes to encode `Option[Product]` at top-level as single struct column. So we can work around the issue that entire top-level row can't be null.

To summarize encoding of `Product` and `Option[Product]`.

For `Product`, 1. at root level, the schema is all fields are flatten it into multiple columns. The `Product ` can't be null, otherwise it throws an exception.

```scala
val df = Seq((1 -> "a"), (2 -> "b")).toDF()
df.printSchema()

root
 |-- _1: integer (nullable = false)
 |-- _2: string (nullable = true)
```

2. At non-root level, `Product` is a struct type column.

```scala
val df = Seq((1, (1 -> "a")), (2, (2 -> "b")), (3, null)).toDF()
df.printSchema()

root
 |-- _1: integer (nullable = false)
 |-- _2: struct (nullable = true)
 |    |-- _1: integer (nullable = false)
 |    |-- _2: string (nullable = true)
```

For `Option[Product]`, 1. it was not supported at root level. After this change, it is a struct type column.

```scala
val df = Seq(Some(1 -> "a"), Some(2 -> "b"), None).toDF()
df.printSchema

root
 |-- value: struct (nullable = true)
 |    |-- _1: integer (nullable = false)
 |    |-- _2: string (nullable = true)
```

2. At non-root level, it is also a struct type column.

```scala
val df = Seq((1, Some(1 -> "a")), (2, Some(2 -> "b")), (3, None)).toDF()
df.printSchema

root
 |-- _1: integer (nullable = false)
 |-- _2: struct (nullable = true)
 |    |-- _1: integer (nullable = false)
 |    |-- _2: string (nullable = true)
```

3. For use case like Aggregator, it was not supported too. After this change, we support to use `Option[Product]` as buffer/output column type.

```scala
val df = Seq(
    OptionBooleanIntData("bob", Some((true, 1))),
    OptionBooleanIntData("bob", Some((false, 2))),
    OptionBooleanIntData("bob", None)).toDF()

val group = df
    .groupBy("name")
    .agg(OptionBooleanIntAggregator("isGood").toColumn.alias("isGood"))
group.printSchema

root
 |-- name: string (nullable = true)
 |-- isGood: struct (nullable = true)
 |    |-- _1: boolean (nullable = false)
 |    |-- _2: integer (nullable = false)
```

The buffer and output type of `OptionBooleanIntAggregator` is both `Option[(Boolean, Int)`.

## How was this patch tested?

Added test.

Closes #21732 from viirya/SPARK-24762.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-26 11:13:28 +08:00
gatorsmile 94145786a5 [SPARK-25908][SQL][FOLLOW-UP] Add back unionAll
## What changes were proposed in this pull request?
This PR is to add back `unionAll`, which is widely used. The name is also consistent with our ANSI SQL. We also have the corresponding `intersectAll` and `exceptAll`, which were introduced in Spark 2.4.

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

Closes #23131 from gatorsmile/addBackUnionAll.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-11-25 15:53:07 -08:00
Katrin Leinweber c5daccb1da [MINOR] Update all DOI links to preferred resolver
## What changes were proposed in this pull request?

The DOI foundation recommends [this new resolver](https://www.doi.org/doi_handbook/3_Resolution.html#3.8). Accordingly, this PR re`sed`s all static DOI links ;-)

## How was this patch tested?

It wasn't, since it seems as safe as a "[typo fix](https://spark.apache.org/contributing.html)".

In case any of the files is included from other projects, and should be updated there, please let me know.

Closes #23129 from katrinleinweber/resolve-DOIs-securely.

Authored-by: Katrin Leinweber <9948149+katrinleinweber@users.noreply.github.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-25 17:43:55 -06:00
hyukjinkwon 41d5aaec84 [SPARK-26148][PYTHON][TESTS] Increases default parallelism in PySpark tests to speed up
## What changes were proposed in this pull request?

This PR proposes to increase parallelism in PySpark tests to speed up from 4 to 8.

It decreases the elapsed time from

https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99163/consoleFull
Tests passed in 1770 seconds

to

https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99186/testReport/
Tests passed in 1027 seconds

## How was this patch tested?

Jenkins tests

Closes #23111 from HyukjinKwon/parallelism.

Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-11-26 00:26:24 +09:00
Lee moon soo eea4a0330b
[MINOR][K8S] Invalid property "spark.driver.pod.name" is referenced in docs.
## What changes were proposed in this pull request?

"Running on Kubernetes" references `spark.driver.pod.name` few places, and it should be `spark.kubernetes.driver.pod.name`.

## How was this patch tested?
See changes

Closes #23133 from Leemoonsoo/fix-driver-pod-name-prop.

Authored-by: Lee moon soo <moon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-24 16:09:13 -08:00
Gengliang Wang 0f56977f8c [SPARK-26156][WEBUI] Revise summary section of stage page
## What changes were proposed in this pull request?
In the summary section of stage page:
![image](https://user-images.githubusercontent.com/1097932/48935518-ebef2b00-ef42-11e8-8672-eaa4cac92c5e.png)

1.  the following metrics names can be revised:
Output => Output Size / Records
Shuffle Read: => Shuffle Read Size / Records
Shuffle Write => Shuffle Write Size / Records

After changes, the names are more clear, and consistent with the other names in the same page.

2. The associated job id URL should not contain the 3 tails spaces. Reduce the number of spaces to one, and exclude the space from link. This is consistent with SQL execution page.

## How was this patch tested?

Manual check:
![image](https://user-images.githubusercontent.com/1097932/48935538-f7425680-ef42-11e8-8b2a-a4f388d3ea52.png)

Closes #23125 from gengliangwang/reviseStagePage.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-24 09:12:05 -06:00
liuxian 7f5f7a967d [SPARK-25786][CORE] If the ByteBuffer.hasArray is false , it will throw UnsupportedOperationException for Kryo
## What changes were proposed in this pull request?
`deserialize` for kryo,  the type of input parameter is ByteBuffer, if it is not backed by an accessible byte array. it will throw `UnsupportedOperationException`

Exception Info:
```
java.lang.UnsupportedOperationException was thrown.
java.lang.UnsupportedOperationException
    at java.nio.ByteBuffer.array(ByteBuffer.java:994)
    at org.apache.spark.serializer.KryoSerializerInstance.deserialize(KryoSerializer.scala:362)
```

## How was this patch tested?

Added a unit test

Closes #22779 from 10110346/InputStreamKryo.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-24 09:10:15 -06:00
Reynold Xin de84899204 [SPARK-26140] Enable custom metrics implementation in shuffle reader
## What changes were proposed in this pull request?
This patch defines an internal Spark interface for reporting shuffle metrics and uses that in shuffle reader. Before this patch, shuffle metrics is tied to a specific implementation (using a thread local temporary data structure and accumulators). After this patch, callers that define their own shuffle RDDs can create a custom metrics implementation.

With this patch, we would be able to create a better metrics for the SQL layer, e.g. reporting shuffle metrics in the SQL UI, for each exchange operator.

Note that I'm separating read side and write side implementations, as they are very different, to simplify code review. Write side change is at https://github.com/apache/spark/pull/23106

## How was this patch tested?
No behavior change expected, as it is a straightforward refactoring. Updated all existing test cases.

Closes #23105 from rxin/SPARK-26140.

Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-11-23 14:14:21 -08:00
Juliusz Sompolski ecb785f4e4 [SPARK-26038] Decimal toScalaBigInt/toJavaBigInteger for decimals not fitting in long
## What changes were proposed in this pull request?

Fix Decimal `toScalaBigInt` and `toJavaBigInteger` used to only work for decimals not fitting long.

## How was this patch tested?

Added test to DecimalSuite.

Closes #23022 from juliuszsompolski/SPARK-26038.

Authored-by: Juliusz Sompolski <julek@databricks.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
2018-11-23 21:08:06 +01:00
Maxim Gekk 8e8d1177e6 [SPARK-26108][SQL] Support custom lineSep in CSV datasource
## What changes were proposed in this pull request?

In the PR,  I propose new options for CSV datasource - `lineSep` similar to Text and JSON datasource. The option allows to specify custom line separator of maximum length of 2 characters (because of a restriction in `uniVocity` parser). New option can be used in reading and writing CSV files.

## How was this patch tested?

Added a few tests with custom `lineSep` for enabled/disabled `multiLine` in read as well as tests in write. Also I added roundtrip tests.

Closes #23080 from MaxGekk/csv-line-sep.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-11-24 00:50:20 +09:00
caoxuewen 466d011d35 [SPARK-26117][CORE][SQL] use SparkOutOfMemoryError instead of OutOfMemoryError when catch exception
## What changes were proposed in this pull request?

the pr #20014 which introduced `SparkOutOfMemoryError` to avoid killing the entire executor when an `OutOfMemoryError `is thrown.
so apply for memory using `MemoryConsumer. allocatePage `when  catch exception, use `SparkOutOfMemoryError `instead of `OutOfMemoryError`

## How was this patch tested?
N / A

Closes #23084 from heary-cao/SparkOutOfMemoryError.

Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-23 21:12:25 +08:00
Shixiong Zhu 92fc0a8f96 [SPARK-26069][TESTS][FOLLOWUP] Add another possible error message
## What changes were proposed in this pull request?

`org.apache.spark.network.RpcIntegrationSuite.sendRpcWithStreamFailures` is still flaky and here is error message:

```
sbt.ForkMain$ForkError: java.lang.AssertionError: Got a non-empty set [Failed to send RPC RPC 8249697863992194475 to /172.17.0.2:41177: java.io.IOException: Broken pipe]
	at org.junit.Assert.fail(Assert.java:88)
	at org.junit.Assert.assertTrue(Assert.java:41)
	at org.apache.spark.network.RpcIntegrationSuite.assertErrorAndClosed(RpcIntegrationSuite.java:389)
	at org.apache.spark.network.RpcIntegrationSuite.sendRpcWithStreamFailures(RpcIntegrationSuite.java:347)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
	at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
	at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
	at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
	at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
	at org.junit.runners.Suite.runChild(Suite.java:128)
	at org.junit.runners.Suite.runChild(Suite.java:27)
	at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
	at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
	at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
	at org.junit.runner.JUnitCore.run(JUnitCore.java:137)
	at org.junit.runner.JUnitCore.run(JUnitCore.java:115)
	at com.novocode.junit.JUnitRunner$1.execute(JUnitRunner.java:132)
	at sbt.ForkMain$Run$2.call(ForkMain.java:296)
	at sbt.ForkMain$Run$2.call(ForkMain.java:286)
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
```

This happened when the second RPC message was being sent but the connection was closed at the same time.

## How was this patch tested?

Jenkins

Closes #23109 from zsxwing/SPARK-26069-2.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-23 06:18:44 -06:00
Dongjoon Hyun 1d3dd58d21
[SPARK-25954][SS][FOLLOWUP][TEST-MAVEN] Add Zookeeper 3.4.7 test dependency to Kafka modules
## What changes were proposed in this pull request?

This is a followup of #23099 . After upgrading to Kafka 2.1.0, maven test fails due to Zookeeper test dependency while sbt test succeeds.

- [sbt test on master branch](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.7/5203/)
- [maven test on master branch](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/5653/)

The root cause is that the embedded Kafka server is using [Zookeepr 3.4.7 API](https://zookeeper.apache.org/doc/r3.4.7/api/org/apache/zookeeper/AsyncCallback.MultiCallback.html
) while Apache Spark provides Zookeeper 3.4.6. This PR adds a test dependency.

```
KafkaMicroBatchV2SourceSuite:
*** RUN ABORTED ***
...
org.apache.spark.sql.kafka010.KafkaTestUtils.setupEmbeddedKafkaServer(KafkaTestUtils.scala:123)
  ...
  Cause: java.lang.ClassNotFoundException: org.apache.zookeeper.AsyncCallback$MultiCallback
  at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
  at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
  at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:331)
  at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
  at kafka.zk.KafkaZkClient$.apply(KafkaZkClient.scala:1693)
  at kafka.server.KafkaServer.createZkClient$1(KafkaServer.scala:348)
  at kafka.server.KafkaServer.initZkClient(KafkaServer.scala:372)
  at kafka.server.KafkaServer.startup(KafkaServer.scala:202)
  at org.apache.spark.sql.kafka010.KafkaTestUtils.$anonfun$setupEmbeddedKafkaServer$2(KafkaTestUtils.scala:120)
  at org.apache.spark.sql.kafka010.KafkaTestUtils.$anonfun$setupEmbeddedKafkaServer$2$adapted(KafkaTestUtils.scala:116)
  ...
```

## How was this patch tested?

Pass the maven Jenkins test.

Closes #23119 from dongjoon-hyun/SPARK-25954-2.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-22 22:45:08 -08:00
Alon Doron 0ec7b99ea2 [SPARK-26021][SQL] replace minus zero with zero in Platform.putDouble/Float
GROUP BY treats -0.0 and 0.0 as different values which is unlike hive's behavior.
In addition current behavior with codegen is unpredictable (see example in JIRA ticket).

## What changes were proposed in this pull request?

In Platform.putDouble/Float() checking if the value is -0.0, and if so replacing with 0.0.
This is used by UnsafeRow so it won't have -0.0 values.

## How was this patch tested?

Added tests

Closes #23043 from adoron/adoron-spark-26021-replace-minus-zero-with-zero.

Authored-by: Alon Doron <adoron@palantir.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-23 08:55:00 +08:00
jerryshao 76aae7f1fd
[SPARK-24553][UI][FOLLOWUP] Fix unnecessary UI redirect
## What changes were proposed in this pull request?

This PR is a follow-up PR of #21600 to fix the unnecessary UI redirect.

## How was this patch tested?

Local verification

Closes #23116 from jerryshao/SPARK-24553.

Authored-by: jerryshao <jerryshao@tencent.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-22 14:54:00 -08:00
Dongjoon Hyun 1d766f0e22
[SPARK-26144][BUILD] build/mvn should detect scala.version based on scala.binary.version
## What changes were proposed in this pull request?

Currently, `build/mvn` downloads and uses **Scala 2.12.7** in `Scala-2.11` Jenkins job. The root cause is `build/mvn` got the first match from `pom.xml` blindly.

- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7-ubuntu-scala-2.11/6/consoleFull
```
exec: curl -s -L https://downloads.lightbend.com/zinc/0.3.15/zinc-0.3.15.tgz
exec: curl -s -L https://downloads.lightbend.com/scala/2.12.7/scala-2.12.7.tgz
exec: curl -s -L https://www.apache.org/dyn/closer.lua?action=download&filename=/maven/maven-3/3.5.4/binaries/apache-maven-3.5.4-bin.tar.gz
```

## How was this patch tested?

Manual.
```
$ build/mvn clean
exec: curl --progress-bar -L https://downloads.lightbend.com/scala/2.12.7/scala-2.12.7.tgz
...
$ git clean -fdx
$ dev/change-scala-version.sh 2.11
$ build/mvn clean
exec: curl --progress-bar -L https://downloads.lightbend.com/scala/2.11.12/scala-2.11.12.tgz
```

Closes #23118 from dongjoon-hyun/SPARK-26144.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-22 14:49:41 -08:00
oraviv d81d95a7e8 [SPARK-19368][MLLIB] BlockMatrix.toIndexedRowMatrix() optimization for sparse matrices
## What changes were proposed in this pull request?

Optimization [SPARK-12869] was made for dense matrices but caused great performance issue for sparse matrices because manipulating them is very inefficient. When manipulating sparse matrices in Breeze we better use VectorBuilder.

## How was this patch tested?

checked it against a use case that we have that after moving to Spark 2 took 6.5 hours instead of 20 mins. After the change it is back to 20 mins again.

Closes #16732 from uzadude/SparseVector_optimization.

Authored-by: oraviv <oraviv@paypal.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-22 15:48:01 -06:00
Marco Gaido dd8c179c28 [SPARK-25867][ML] Remove KMeans computeCost
## What changes were proposed in this pull request?

The PR removes the deprecated method `computeCost` of `KMeans`.

## How was this patch tested?

NA

Closes #22875 from mgaido91/SPARK-25867.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-22 15:45:25 -06:00
Huon Wilson aeda76e2b7 [GRAPHX] Remove unused variables left over by previous refactoring.
## What changes were proposed in this pull request?

Some variables were previously used for indexing the routing table's backing
array, but that indexing now happens elsewhere, and so the variables aren't needed.

## How was this patch tested?

Unit tests.

(This contribution is my original work and I license the work to Spark under its open source license.)

Closes #23112 from huonw/remove-unused-variables.

Authored-by: Huon Wilson <Huon.Wilson@data61.csiro.au>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-22 15:43:04 -06:00
Reynold Xin ab00533490
[SPARK-26129][SQL] edge behavior for QueryPlanningTracker.topRulesByTime - followup patch
## What changes were proposed in this pull request?
This is an addendum patch for SPARK-26129 that defines the edge case behavior for QueryPlanningTracker.topRulesByTime.

## How was this patch tested?
Added unit tests for each behavior.

Closes #23110 from rxin/SPARK-26129-1.

Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-22 02:27:06 -08:00