## What changes were proposed in this pull request?
This PR refactors `ShuffleMapStage` and `MapOutputTracker` in order to simplify the management of `MapStatuses`, reduce driver memory consumption, and remove a potential source of scheduler correctness bugs.
### Background
In Spark there are currently two places where MapStatuses are tracked:
- The `MapOutputTracker` maintains an `Array[MapStatus]` storing a single location for each map output. This mapping is used by the `DAGScheduler` for determining reduce-task locality preferences (when locality-aware reduce task scheduling is enabled) and is also used to serve map output locations to executors / tasks.
- Each `ShuffleMapStage` also contains a mapping of `Array[List[MapStatus]]` which holds the complete set of locations where each map output could be available. This mapping is used to determine which map tasks need to be run when constructing `TaskSets` for the stage.
This duplication adds complexity and creates the potential for certain types of correctness bugs. Bad things can happen if these two copies of the map output locations get out of sync. For instance, if the `MapOutputTracker` is missing locations for a map output but `ShuffleMapStage` believes that locations are available then tasks will fail with `MetadataFetchFailedException` but `ShuffleMapStage` will not be updated to reflect the missing map outputs, leading to situations where the stage will be reattempted (because downstream stages experienced fetch failures) but no task sets will be launched (because `ShuffleMapStage` thinks all maps are available).
I observed this behavior in a real-world deployment. I'm still not quite sure how the state got out of sync in the first place, but we can completely avoid this class of bug if we eliminate the duplicate state.
### Why we only need to track a single location for each map output
I think that storing an `Array[List[MapStatus]]` in `ShuffleMapStage` is unnecessary.
First, note that this adds memory/object bloat to the driver we need one extra `List` per task. If you have millions of tasks across all stages then this can add up to be a significant amount of resources.
Secondly, I believe that it's extremely uncommon that these lists will ever contain more than one entry. It's not impossible, but is very unlikely given the conditions which must occur for that to happen:
- In normal operation (no task failures) we'll only run each task once and thus will have at most one output.
- If speculation is enabled then it's possible that we'll have multiple attempts of a task. The TaskSetManager will [kill duplicate attempts of a task](04901dd03a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala (L717)) after a task finishes successfully, reducing the likelihood that both the original and speculated task will successfully register map outputs.
- There is a [comment in `TaskSetManager`](04901dd03a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala (L113)) which suggests that running tasks are not killed if a task set becomes a zombie. However:
- If the task set becomes a zombie due to the job being cancelled then it doesn't matter whether we record map outputs.
- If the task set became a zombie because of a stage failure (e.g. the map stage itself had a fetch failure from an upstream match stage) then I believe that the "failedEpoch" will be updated which may cause map outputs from still-running tasks to [be ignored](04901dd03a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala (L1213)). (I'm not 100% sure on this point, though).
- Even if you _do_ manage to record multiple map outputs for a stage, only a single map output is reported to / tracked by the MapOutputTracker. The only situation where the additional output locations could actually be read or used would be if a task experienced a `FetchFailure` exception. The most likely cause of a `FetchFailure` exception is an executor lost, which will have most likely caused the loss of several map tasks' output, so saving on potential re-execution of a single map task isn't a huge win if we're going to have to recompute several other lost map outputs from other tasks which ran on that lost executor. Also note that the re-population of MapOutputTracker state from state in the ShuffleMapTask only happens after the reduce stage has failed; the additional location doesn't help to prevent FetchFailures but, instead, can only reduce the amount of work when recomputing missing parent stages.
Given this, this patch chooses to do away with tracking multiple locations for map outputs and instead stores only a single location. This change removes the main distinction between the `ShuffleMapTask` and `MapOutputTracker`'s copies of this state, paving the way for storing it only in the `MapOutputTracker`.
### Overview of other changes
- Significantly simplified the cache / lock management inside of the `MapOutputTrackerMaster`:
- The old code had several parallel `HashMap`s which had to be guarded by maps of `Object`s which were used as locks. This code was somewhat complicated to follow.
- The new code uses a new `ShuffleStatus` class to group together all of the state associated with a particular shuffle, including cached serialized map statuses, significantly simplifying the logic.
- Moved more code out of the shared `MapOutputTracker` abstract base class and into the `MapOutputTrackerMaster` and `MapOutputTrackerWorker` subclasses. This makes it easier to reason about which functionality needs to be supported only on the driver or executor.
- Removed a bunch of code from the `DAGScheduler` which was used to synchronize information from the `MapOutputTracker` to `ShuffleMapStage`.
- Added comments to clarify the role of `MapOutputTrackerMaster`'s `epoch` in invalidating executor-side shuffle map output caches.
I will comment on these changes via inline GitHub review comments.
/cc hvanhovell and rxin (whom I discussed this with offline), tgravescs (who recently worked on caching of serialized MapOutputStatuses), and kayousterhout and markhamstra (for scheduler changes).
## How was this patch tested?
Existing tests. I purposely avoided making interface / API which would require significant updates or modifications to test code.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#17955 from JoshRosen/map-output-tracker-rewrite.
## What changes were proposed in this pull request?
Add support for specific Java `List` subtypes in deserialization as well as a generic implicit encoder.
All `List` subtypes are supported by using either the size-specifying constructor (one `int` parameter) or the default constructor.
Interfaces/abstract classes use the following implementations:
* `java.util.List`, `java.util.AbstractList` or `java.util.AbstractSequentialList` => `java.util.ArrayList`
## How was this patch tested?
```bash
build/mvn -DskipTests clean package && dev/run-tests
```
Additionally in Spark shell:
```
scala> val jlist = new java.util.LinkedList[Int]; jlist.add(1)
jlist: java.util.LinkedList[Int] = [1]
res0: Boolean = true
scala> Seq(jlist).toDS().map(_.element()).collect()
res1: Array[Int] = Array(1)
```
Author: Michal Senkyr <mike.senkyr@gmail.com>
Closes#18009 from michalsenkyr/dataset-java-lists.
## What changes were proposed in this pull request?
Currently, hive's stats are read into `CatalogStatistics`, while spark's stats are also persisted through `CatalogStatistics`. As a result, hive's stats can be unexpectedly propagated into spark' stats.
For example, for a catalog table, we read stats from hive, e.g. "totalSize" and put it into `CatalogStatistics`. Then, by using "ALTER TABLE" command, we will store the stats in `CatalogStatistics` into metastore as spark's stats (because we don't know whether it's from spark or not). But spark's stats should be only generated by "ANALYZE" command. This is unexpected from this command.
Secondly, now that we have spark's stats in metastore, after inserting new data, although hive updated "totalSize" in metastore, we still cannot get the right `sizeInBytes` in `CatalogStatistics`, because we respect spark's stats (should not exist) over hive's stats.
A running example is shown in [JIRA](https://issues.apache.org/jira/browse/SPARK-21031).
To fix this, we add a new method `alterTableStats` to store spark's stats, and let `alterTable` keep existing stats.
## How was this patch tested?
Added new tests.
Author: Zhenhua Wang <wzh_zju@163.com>
Closes#18248 from wzhfy/separateHiveStats.
## What changes were proposed in this pull request?
I fixed a typo in the Scaladoc for the method `def struct(cols: Column*): Column`. 'retained' was misspelt as 'remained'.
## How was this patch tested?
Before:
Creates a new struct column.
If the input column is a column in a `DataFrame`, or a derived column expression
that is named (i.e. aliased), its name would be **remained** as the StructField's name,
otherwise, the newly generated StructField's name would be auto generated as
`col` with a suffix `index + 1`, i.e. col1, col2, col3, ...
After:
Creates a new struct column.
If the input column is a column in a `DataFrame`, or a derived column expression
that is named (i.e. aliased), its name would be **retained** as the StructField's name,
otherwise, the newly generated StructField's name would be auto generated as
`col` with a suffix `index + 1`, i.e. col1, col2, col3, ...
Author: sujithjay <sujith@logistimo.com>
Closes#18254 from sujithjay/fix-typo.
## What changes were proposed in this pull request?
clean up after big test move
## How was this patch tested?
unit tests, jenkins
Author: Felix Cheung <felixcheung_m@hotmail.com>
Closes#18267 from felixcheung/rtestset2.
## What changes were proposed in this pull request?
Update hadoop-2.7 profile's curator version to 2.7.1, more see [SPARK-13933](https://issues.apache.org/jira/browse/SPARK-13933).
## How was this patch tested?
manual tests
Author: Yuming Wang <wgyumg@gmail.com>
Closes#18247 from wangyum/SPARK-13933.
## What changes were proposed in this pull request?
This PR proposes to stop `ReceiverTracker` to close `WriteAheadLog` whenever it is and make `WriteAheadLog` and its implementations idempotent.
## How was this patch tested?
Added a test in `WriteAheadLogSuite`. Note that the added test looks passing even if it closes twice (namely even without the changes in `FileBasedWriteAheadLog` and `BatchedWriteAheadLog`. It looks both are already idempotent but this is a rather sanity check.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#18224 from HyukjinKwon/streaming-closing.
## What changes were proposed in this pull request?
Add Mesos labels support to the Spark Dispatcher
## How was this patch tested?
unit tests
Author: Michael Gummelt <mgummelt@mesosphere.io>
Closes#18220 from mgummelt/SPARK-21000-dispatcher-labels.
## What changes were proposed in this pull request?
Move all existing tests to non-installed directory so that it will never run by installing SparkR package
For a follow-up PR:
- remove all skip_on_cran() calls in tests
- clean up test timer
- improve or change basic tests that do run on CRAN (if anyone has suggestion)
It looks like `R CMD build pkg` will still put pkg\tests (ie. the full tests) into the source package but `R CMD INSTALL` on such source package does not install these tests (and so `R CMD check` does not run them)
## How was this patch tested?
- [x] unit tests, Jenkins
- [x] AppVeyor
- [x] make a source package, install it, `R CMD check` it - verify the full tests are not installed or run
Author: Felix Cheung <felixcheung_m@hotmail.com>
Closes#18264 from felixcheung/rtestset.
## What changes were proposed in this pull request?
add more datatype for some unit tests
## How was this patch tested?
unit tests
Author: liuxian <liu.xian3@zte.com.cn>
Closes#17880 from 10110346/wip_lx_0506.
### What changes were proposed in this pull request?
The precision and scale of decimal values are wrong when the input is BigDecimal between -1.0 and 1.0.
The BigDecimal's precision is the digit count starts from the leftmost nonzero digit based on the [JAVA's BigDecimal definition](https://docs.oracle.com/javase/7/docs/api/java/math/BigDecimal.html). However, our Decimal decision follows the database decimal standard, which is the total number of digits, including both to the left and the right of the decimal point. Thus, this PR is to fix the issue by doing the conversion.
Before this PR, the following queries failed:
```SQL
select 1 > 0.0001
select floor(0.0001)
select ceil(0.0001)
```
### How was this patch tested?
Added test cases.
Author: Xiao Li <gatorsmile@gmail.com>
Closes#18244 from gatorsmile/bigdecimal.
## What changes were proposed in this pull request?
Document Dataset.union is resolution by position, not by name, since this has been a confusing point for a lot of users.
## How was this patch tested?
N/A - doc only change.
Author: Reynold Xin <rxin@databricks.com>
Closes#18256 from rxin/SPARK-21042.
### What changes were proposed in this pull request?
Currently, the unquoted string of a function identifier is being used as the function identifier in the function registry. This could cause the incorrect the behavior when users use `.` in the function names. This PR is to take the `FunctionIdentifier` as the identifier in the function registry.
- Add one new function `createOrReplaceTempFunction` to `FunctionRegistry`
```Scala
final def createOrReplaceTempFunction(name: String, builder: FunctionBuilder): Unit
```
### How was this patch tested?
Add extra test cases to verify the inclusive bug fixes.
Author: Xiao Li <gatorsmile@gmail.com>
Author: gatorsmile <gatorsmile@gmail.com>
Closes#18142 from gatorsmile/fuctionRegistry.
## What changes were proposed in this pull request?
'--driver-cores' standalone or Mesos or YARN in Cluster deploy mode only.So The description of spark-submit about it is not very accurate.
## How was this patch tested?
manual tests
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: guoxiaolong <guo.xiaolong1@zte.com.cn>
Author: 郭小龙 10207633 <guo.xiaolong1@zte.com.cn>
Author: guoxiaolongzte <guo.xiaolong1@zte.com.cn>
Closes#18241 from guoxiaolongzte/SPARK-20997.
the original code cant visit the last element of the"parts" array.
so the v[v.length–1] always equals 0
## What changes were proposed in this pull request?
change the recycle range from (1 to parts.length-1) to (1 to parts.length)
## How was this patch tested?
debug it in eclipse (´〜`*) zzz.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: junzhi lu <452756565@qq.com>
Closes#18237 from masterwugui/patch-1.
## What changes were proposed in this pull request?
Ensure that `HADOOP_CONF_DIR` or `YARN_CONF_DIR` points to the directory which contains the (client side) configuration files for the Hadoop cluster.
These configs are used to write to HDFS and connect to the YARN ResourceManager. The
configuration contained in this directory will be distributed to the YARN cluster so that all
containers used by the application use the same configuration.
Sometimes, `HADOOP_CONF_DIR` is set to the hdfs configuration file path. So, YARN_CONF_DIR should be set to the yarn configuration file path.
My project configuration item of 'spark-env.sh ' is as follows:
![1](https://cloud.githubusercontent.com/assets/26266482/26819987/d4acb814-4ad3-11e7-8458-a21aea57a53d.png)
'HADOOP_CONF_DIR' configuration file path. List the relevant documents below:
![3](https://cloud.githubusercontent.com/assets/26266482/26820116/47b6b9fe-4ad4-11e7-8131-fe07c8d8bc21.png)
'YARN_CONF_DIR' configuration file path. List the relevant documents below:
![2](https://cloud.githubusercontent.com/assets/26266482/26820078/274ad79a-4ad4-11e7-83d4-ff359dbb397c.png)
So, 'Spark-env.sh.template' should add 'YARN_CONF_DIR' configuration instructions.
## How was this patch tested?
manual tests
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: guoxiaolong <guo.xiaolong1@zte.com.cn>
Author: 郭小龙 10207633 <guo.xiaolong1@zte.com.cn>
Author: guoxiaolongzte <guo.xiaolong1@zte.com.cn>
Closes#18212 from guoxiaolongzte/SPARK-20995.
## What changes were proposed in this pull request?
Previously, `RDD.treeAggregate` used `reduceByKey` and `reduce` in its implementation, neither of which technically allows the `seq`/`combOps` to modify and return their first arguments.
This PR uses `foldByKey` and `fold` instead and notes that `aggregate` and `treeAggregate` are semantically identical in the Scala doc.
Note that this had some test failures by unknown reasons. This was actually fixed in e3554605b3.
The root cause was, the `zeroValue` now becomes `AFTAggregator` and it compares `totalCnt` (where the value is actually 0). It starts merging one by one and it keeps returning `this` where `totalCnt` is 0. So, this looks not the bug in the current change.
This is now fixed in the commit. So, this should pass the tests.
## How was this patch tested?
Test case added in `RDDSuite`.
Closes#12217
Author: Joseph K. Bradley <joseph@databricks.com>
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#18198 from HyukjinKwon/SPARK-14408.
## What changes were proposed in this pull request?
This patch adds Coda Hale metrics for instrumenting the `LiveListenerBus` in order to track the number of events received, dropped, and processed. In addition, it adds per-SparkListener-subclass timers to track message processing time. This is useful for identifying when slow third-party SparkListeners cause performance bottlenecks.
See the new `LiveListenerBusMetrics` for a complete description of the new metrics.
## How was this patch tested?
New tests in SparkListenerSuite, including a test to ensure proper counting of dropped listener events.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#18083 from JoshRosen/listener-bus-metrics.
## What changes were proposed in this pull request?
After [SPARK-20067](https://issues.apache.org/jira/browse/SPARK-20067), `DESCRIBE` and `DESCRIBE EXTENDED` shows the following result. This is incompatible with Spark 2.1.1. This PR removes the column header line in case of those command.
**MASTER** and **BRANCH-2.2**
```scala
scala> sql("desc t").show(false)
+----------+---------+-------+
|col_name |data_type|comment|
+----------+---------+-------+
|# col_name|data_type|comment|
|a |int |null |
+----------+---------+-------+
```
**SPARK 2.1.1** and **this PR**
```scala
scala> sql("desc t").show(false)
+--------+---------+-------+
|col_name|data_type|comment|
+--------+---------+-------+
|a |int |null |
+--------+---------+-------+
```
## How was this patch tested?
Pass the Jenkins with the updated test suites.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#18203 from dongjoon-hyun/SPARK-20954.
### What changes were proposed in this pull request?
Before 2.2, we indicate the job was terminated because of `FAILFAST` mode.
```
Malformed line in FAILFAST mode: {"a":{, b:3}
```
If possible, we should keep it. This PR is to unify the error messages.
### How was this patch tested?
Modified the existing messages.
Author: Xiao Li <gatorsmile@gmail.com>
Closes#18196 from gatorsmile/messFailFast.
## What changes were proposed in this pull request?
Add a new property `spark.streaming.kafka.consumer.cache.enabled` that allows users to enable or disable the cache for Kafka consumers. This property can be especially handy in cases where issues like SPARK-19185 get hit, for which there isn't a solution committed yet. By default, the cache is still on, so this change doesn't change any out-of-box behavior.
## How was this patch tested?
Running unit tests
Author: Mark Grover <mark@apache.org>
Author: Mark Grover <grover.markgrover@gmail.com>
Closes#18234 from markgrover/spark-19185.
Signed-off-by: 10087686 <wang.jiaochunzte.com.cn>
## What changes were proposed in this pull request?
When run test("port conflict") case, we need run anotherEnv.shutdown() and anotherEnv.awaitTermination() for free resource.
(Please fill in changes proposed in this fix)
## How was this patch tested?
run RpcEnvSuit.scala Utest
(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: 10087686 <wang.jiaochun@zte.com.cn>
Closes#18226 from wangjiaochun/master.
## What changes were proposed in this pull request?
Fix Java, Scala Dataset examples in scaladoc, which didn't compile.
## How was this patch tested?
Existing compilation/test
Author: Sean Owen <sowen@cloudera.com>
Closes#18215 from srowen/SPARK-20914.
## What changes were proposed in this pull request?
Removed a duplicate case in "SPARK-20854: select hint syntax with expressions"
## How was this patch tested?
Existing tests.
Author: Bogdan Raducanu <bogdan@databricks.com>
Closes#18217 from bogdanrdc/SPARK-20854-2.
## What changes were proposed in this pull request?
`HintInfo.isBroadcastable` is actually not an accurate name, it's used to force the planner to broadcast a plan no matter what the data size is, via the hint mechanism. I think `forceBroadcast` is a better name.
And `isBroadcastable` only have 2 possible values: `Some(true)` and `None`, so we can just use boolean type for it.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#18189 from cloud-fan/stats.
This change adds an abstraction and LevelDB implementation for a key-value
store that will be used to store UI and SHS data.
The interface is described in KVStore.java (see javadoc). Specifics
of the LevelDB implementation are discussed in the javadocs of both
LevelDB.java and LevelDBTypeInfo.java.
Included also are a few small benchmarks just to get some idea of
latency. Because they're too slow for regular unit test runs, they're
disabled by default.
Tested with the included unit tests, and also as part of the overall feature
implementation (including running SHS with hundreds of apps).
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#17902 from vanzin/shs-ng/M1.
There could be test failures because DataStorageStrategy, HiveMetastoreCatalog and also HiveSchemaInferenceSuite were exposed to guava library by directly accessing SessionCatalog's tableRelationCacheg. These failures occur when guava shading is in place.
## What changes were proposed in this pull request?
This change removes those guava exposures by introducing new methods in SessionCatalog and also changing DataStorageStrategy, HiveMetastoreCatalog and HiveSchemaInferenceSuite so that they use those proxy methods.
## How was this patch tested?
Unit tests passed after applying these changes.
Author: Reza Safi <rezasafi@cloudera.com>
Closes#18148 from rezasafi/branch-2.2.
(cherry picked from commit 1388fdd707)
## What changes were proposed in this pull request?
SparkContext should always be stopped after using, thus other tests won't complain that there's only one `SparkContext` can exist.
Author: jinxing <jinxing6042@126.com>
Closes#18204 from jinxing64/SPARK-20985.
## What changes were proposed in this pull request?
The construction of BROADCAST_TIMEOUT conf should take the TimeUnit argument as a TimeoutConf.
Author: Feng Liu <fengliu@databricks.com>
Closes#18208 from liufengdb/fix_timeout.
## What changes were proposed in this pull request?
When stopping StreamingQuery, StreamExecution will set `streamDeathCause` then notify StreamingQueryManager to remove this query. So it's possible that when `q2.exception.isDefined` returns `true`, StreamingQueryManager's active list still has `q2`.
This PR just puts the checks into `eventually` to fix the flaky test.
## How was this patch tested?
Jenkins
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#18180 from zsxwing/SPARK-20957.
## What changes were proposed in this pull request?
In our use case of launching Spark applications via REST APIs (Livy), there's no way for user to specify command line arguments, all Spark configurations are set through configurations map. For "--repositories" because there's no equivalent Spark configuration, so we cannot specify the custom repository through configuration.
So here propose to add "--repositories" equivalent configuration in Spark.
## How was this patch tested?
New UT added.
Author: jerryshao <sshao@hortonworks.com>
Closes#18201 from jerryshao/SPARK-20981.
## What changes were proposed in this pull request?
JIRA: [SPARK-19762](https://issues.apache.org/jira/browse/SPARK-19762)
The larger changes in this patch are:
* Adds a `DifferentiableLossAggregator` trait which is intended to be used as a common parent trait to all Spark ML aggregator classes. It factors out the common methods: `merge, gradient, loss, weight` from the aggregator subclasses.
* Adds a `RDDLossFunction` which is intended to be the only implementation of Breeze's `DiffFunction` necessary in Spark ML, and can be used by all other algorithms. It takes the aggregator type as a type parameter, and maps the aggregator over an RDD. It additionally takes in a optional regularization loss function for applying the differentiable part of regularization.
* Factors out the regularization from the data part of the cost function, and treats regularization as a separate independent cost function which can be evaluated and added to the data cost function.
* Changes `LinearRegression` to use this new hierarchy as a proof of concept.
* Adds the following new namespaces `o.a.s.ml.optim.loss` and `o.a.s.ml.optim.aggregator`
Also note that none of these are public-facing changes. All of these classes are internal to Spark ML and remain that way.
**NOTE: The large majority of the "lines added" and "lines deleted" are simply code moving around or unit tests.**
BTW, I also converted LinearSVC to this framework as a way to prove that this new hierarchy is flexible enough for the other algorithms, but I backed those changes out because the PR is large enough as is.
## How was this patch tested?
Test suites are added for the new components, and some test suites are also added to provide coverage where there wasn't any before.
* DifferentiablLossAggregatorSuite
* LeastSquaresAggregatorSuite
* RDDLossFunctionSuite
* DifferentiableRegularizationSuite
Below are some performance testing numbers. Run on a 6 node virtual cluster with 44 cores and ~110G RAM, the dataset size is about 37G. These are not "large-scale" tests, but we really want to just make sure the iteration times don't increase with this patch. Notably we are doing the regularization a bit differently than before, but that should cost very little. I think there's very little risk otherwise, and these numbers don't show a difference. Of course I'm happy to add more tests as we think it's necessary, but I think the patch is ready for review now.
**Note:** timings are best of 3 runs.
| | numFeatures | numPoints | maxIter | regParam | elasticNetParam | SPARK-19762 (sec) | master (sec) |
|----|---------------|-------------|-----------|------------|-------------------|---------------------|----------------|
| 0 | 5000 | 1e+06 | 30 | 0 | 0 | 129.594 | 131.153 |
| 1 | 5000 | 1e+06 | 30 | 0.1 | 0 | 135.54 | 136.327 |
| 2 | 5000 | 1e+06 | 30 | 0.01 | 0.5 | 135.148 | 129.771 |
| 3 | 50000 | 100000 | 30 | 0 | 0 | 145.764 | 144.096 |
## Follow ups
If this design is accepted, we will convert the other ML algorithms that use this aggregator pattern to this new hierarchy in follow up PRs.
Author: sethah <seth.hendrickson16@gmail.com>
Author: sethah <shendrickson@cloudera.com>
Closes#17094 from sethah/ml_aggregators.
## What changes were proposed in this pull request?
Destroy broadcasted centers after computing cost
## How was this patch tested?
existing tests
Author: Zheng RuiFeng <ruifengz@foxmail.com>
Closes#18152 from zhengruifeng/destroy_kmeans_model.
## What changes were proposed in this pull request?
This pull request fix the TaskScheulerImpl bug in some condition.
Detail see:
https://issues.apache.org/jira/browse/SPARK-20945
(Please fill in changes proposed in this fix)
## How was this patch tested?
manual tests
(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: liupengcheng <liupengcheng@xiaomi.com>
Author: PengchengLiu <pengchengliu_bupt@163.com>
Closes#18171 from liupc/Fix-tid-key-not-found-in-TaskSchedulerImpl.
## What changes were proposed in this pull request?
As the first step of https://issues.apache.org/jira/browse/SPARK-20960 , to make `ColumnVector` public, this PR generalize `ColumnVector.dictionary` to not couple with parquet.
## How was this patch tested?
existing tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#18183 from cloud-fan/dictionary.
## What changes were proposed in this pull request?
Fixes a typo: `and` -> `an`
## How was this patch tested?
Not at all.
Author: Wieland Hoffmann <mineo@users.noreply.github.com>
Closes#17759 from mineo/patch-1.
## What changes were proposed in this pull request?
1. add `assert(resolve(before) === after)` to check before and after in test of resolveURI.
the function `assertResolves(before: String, after: String)` have two params, it means we should check the before value whether equals the after value which we want.
e.g. the after value of Utils.resolveURI("hdfs:///root/spark.jar#app.jar").toString should be "hdfs:///root/spark.jar#app.jar" rather than "hdfs:/root/spark.jar#app.jar". we need `assert(resolve(before) === after)` to make it more safe.
2. identify the cases between resolveURI and resolveURIs.
3. delete duplicate cases and some small fix make this suit more clear.
## How was this patch tested?
unit tests
Author: zuotingbing <zuo.tingbing9@zte.com.cn>
Closes#18158 from zuotingbing/spark-UtilsSuite.
## What changes were proposed in this pull request?
Remove extraneous logging.
## How was this patch tested?
Unit tests pass.
Author: David Eis <deis@bloomberg.net>
Closes#18188 from davideis/fix-test.
## What changes were proposed in this pull request?
Allow fill/replace of NAs with booleans, both in Python and Scala
## How was this patch tested?
Unit tests, doctests
This PR is original work from me and I license this work to the Spark project
Author: Ruben Berenguel Montoro <ruben@mostlymaths.net>
Author: Ruben Berenguel <ruben@mostlymaths.net>
Closes#18164 from rberenguel/SPARK-19732-fillna-bools.
## What changes were proposed in this pull request?
REPL module depends on SQL module, so we should run REPL tests if SQL module has code changes.
## How was this patch tested?
N/A
Author: Wenchen Fan <wenchen@databricks.com>
Closes#18191 from cloud-fan/test.
## What changes were proposed in this pull request?
Usually when using explain cost command, users want to see the stats of plan. Since stats is only showed in optimized plan, it is more direct and convenient to include only optimized plan and physical plan in the output.
## How was this patch tested?
Enhanced existing test.
Author: Zhenhua Wang <wzh_zju@163.com>
Closes#18190 from wzhfy/simplifyExplainCost.
### What changes were proposed in this pull request?
1. The description of `spark.sql.files.ignoreCorruptFiles` is not accurate. When the file does not exist, we will issue the error message.
```
org.apache.spark.sql.AnalysisException: Path does not exist: file:/nonexist/path;
```
2. `spark.sql.columnNameOfCorruptRecord` also affects the CSV format. The current description only mentions JSON format.
### How was this patch tested?
N/A
Author: Xiao Li <gatorsmile@gmail.com>
Closes#18184 from gatorsmile/updateMessage.
## What changes were proposed in this pull request?
In [this line](f7cf2096fd/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala (L128)), it uses the `executorId` string received from executors and finally it will go into `TaskUIData`. As deserializing the `executorId` string will always create a new instance, we have a lot of duplicated string instances.
This PR does a String interning for TaskUIData to reduce the memory usage.
## How was this patch tested?
Manually test using `bin/spark-shell --master local-cluster[6,1,1024]`. Test codes:
```
for (_ <- 1 to 10) { sc.makeRDD(1 to 1000, 1000).count() }
Thread.sleep(2000)
val l = sc.getClass.getMethod("jobProgressListener").invoke(sc).asInstanceOf[org.apache.spark.ui.jobs.JobProgressListener]
org.apache.spark.util.SizeEstimator.estimate(l.stageIdToData)
```
This PR reduces the size of `stageIdToData` from 3487280 to 3009744 (86.3%) in the above case.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#18177 from zsxwing/SPARK-20955.
## What changes were proposed in this pull request?
The current conf setting logic is a little complex and has duplication, this PR simplifies it.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#18172 from cloud-fan/session.