## What changes were proposed in this pull request?
This PR tries to fix all typos in all markdown files under `docs` module,
and fixes similar typos in other comments, too.
## How was the this patch tested?
manual tests.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#11300 from dongjoon-hyun/minor_fix_typos.
## What changes were proposed in this pull request?
This PR removes the support of SIMR, since SIMR is not actively used and maintained for a long time, also is not supported from `SparkSubmit`, so here propose to remove it.
## How was the this patch tested?
This patch is tested locally by running unit tests.
Author: jerryshao <sshao@hortonworks.com>
Closes#11296 from jerryshao/SPARK-13426.
## What changes were proposed in this pull request?
`JobWaiter.taskSucceeded` will be called for each task. When `resultHandler` throws an exception, `taskSucceeded` will also throw it for each task. DAGScheduler just catches it and reports it like this:
```Scala
try {
job.listener.taskSucceeded(rt.outputId, event.result)
} catch {
case e: Exception =>
// TODO: Perhaps we want to mark the resultStage as failed?
job.listener.jobFailed(new SparkDriverExecutionException(e))
}
```
Therefore `JobWaiter.jobFailed` may be called multiple times.
So `JobWaiter.jobFailed` should use `Promise.tryFailure` instead of `Promise.failure` because the latter one doesn't support calling multiple times.
## How was the this patch tested?
Jenkins tests.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#11280 from zsxwing/SPARK-13408.
`TaskMetrics.fromAccumulatorUpdates()` can fail if accumulators have been garbage-collected on the driver. To guard against this, this patch introduces `ListenerTaskMetrics`, a subclass of `TaskMetrics` which is used only in `TaskMetrics.fromAccumulatorUpdates()` and which eliminates the need to access the original accumulators on the driver.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#11276 from JoshRosen/accum-updates-fix.
## What changes were proposed in this pull request?
Fix some comparisons between unequal types that cause IJ warnings and in at least one case a likely bug (TaskSetManager)
## How was the this patch tested?
Running Jenkins tests
Author: Sean Owen <sowen@cloudera.com>
Closes#11253 from srowen/SPARK-13371.
See http://openjdk.java.net/jeps/223 for more information about the JDK 9 version string scheme.
Author: Claes Redestad <claes.redestad@gmail.com>
Closes#11160 from cl4es/master.
Overrode the start() method, which was previously starting a thread causing a race condition. I believe this should fix the flaky test.
Author: Michael Gummelt <mgummelt@mesosphere.io>
Closes#11164 from mgummelt/fix_mesos_tests.
This JIRA is related to
https://github.com/apache/spark/pull/5852
Had to do some minor rework and test to make sure it
works with current version of spark.
Author: Sanket <schintap@untilservice-lm>
Closes#10838 from redsanket/limit-outbound-connections.
When the HistoryServer is showing an incomplete app, it needs to check if there is a newer version of the app available. It does this by checking if a version of the app has been loaded with a larger *filesize*. If so, it detaches the current UI, attaches the new one, and redirects back to the same URL to show the new UI.
https://issues.apache.org/jira/browse/SPARK-7889
Author: Steve Loughran <stevel@hortonworks.com>
Author: Imran Rashid <irashid@cloudera.com>
Closes#11118 from squito/SPARK-7889-alternate.
The "getPersistentRDDs()" is a useful API of SparkContext to get cached RDDs. However, the JavaSparkContext does not have this API.
Add a simple getPersistentRDDs() to get java.util.Map<Integer, JavaRDD> for Java users.
Author: Junyang <fly.shenjy@gmail.com>
Closes#10978 from flyjy/master.
Remove spark.closure.serializer option and use JavaSerializer always
CC andrewor14 rxin I see there's a discussion in the JIRA but just thought I'd offer this for a look at what the change would be.
Author: Sean Owen <sowen@cloudera.com>
Closes#11150 from srowen/SPARK-12414.
This is the next iteration of tnachen's previous PR: https://github.com/apache/spark/pull/4027
In that PR, we resolved with andrewor14 and pwendell to implement the Mesos scheduler's support of `spark.executor.cores` to be consistent with YARN and Standalone. This PR implements that resolution.
This PR implements two high-level features. These two features are co-dependent, so they're implemented both here:
- Mesos support for spark.executor.cores
- Multiple executors per slave
We at Mesosphere have been working with Typesafe on a Spark/Mesos integration test suite: https://github.com/typesafehub/mesos-spark-integration-tests, which passes for this PR.
The contribution is my original work and I license the work to the project under the project's open source license.
Author: Michael Gummelt <mgummelt@mesosphere.io>
Closes#10993 from mgummelt/executor_sizing.
This PR improve the lookup of BytesToBytesMap by:
1. Generate code for calculate the hash code of grouping keys.
2. Do not use MemoryLocation, fetch the baseObject and offset for key and value directly (remove the indirection).
Author: Davies Liu <davies@databricks.com>
Closes#11010 from davies/gen_map.
Additional changes to #10835, mainly related to style and visibility. This patch also adds back a few deprecated methods for backward compatibility.
Author: Andrew Or <andrew@databricks.com>
Closes#10958 from andrewor14/task-metrics-to-accums-followups.
There is a bug when we try to grow the buffer, OOM is ignore wrongly (the assert also skipped by JVM), then we try grow the array again, this one will trigger spilling free the current page, the current record we inserted will be invalid.
The root cause is that JVM has less free memory than MemoryManager thought, it will OOM when allocate a page without trigger spilling. We should catch the OOM, and acquire memory again to trigger spilling.
And also, we could not grow the array in `insertRecord` of `InMemorySorter` (it was there just for easy testing).
Author: Davies Liu <davies@databricks.com>
Closes#11095 from davies/fix_expand.
Trivial search-and-replace to eliminate deprecation warnings in Scala 2.11.
Also works with 2.10
Author: Jakob Odersky <jakob@odersky.com>
Closes#11085 from jodersky/SPARK-13171.
These were ignored because they are incorrectly written; they don't actually trigger stage retries, which is what the tests are testing. These tests are now rewritten to induce stage retries through fetch failures.
Note: there were 2 tests before and now there's only 1. What happened? It turns out that the case where we only resubmit a subset of of the original missing partitions is very difficult to simulate in tests without potentially introducing flakiness. This is because the `DAGScheduler` removes all map outputs associated with a given executor when this happens, and we will need multiple executors to trigger this case, and sometimes the scheduler still removes map outputs from all executors.
Author: Andrew Or <andrew@databricks.com>
Closes#10969 from andrewor14/unignore-accum-test.
Currently the Master would always set an application's initial executor limit to infinity. If the user specified `spark.dynamicAllocation.initialExecutors`, the config would not take effect. This is similar to #11047 but for standalone mode.
Author: Andrew Or <andrew@databricks.com>
Closes#11054 from andrewor14/standalone-da-initial.
Building with scala 2.11 results in the warning trait SynchronizedBuffer in package mutable is deprecated: Synchronization via traits is deprecated as it is inherently unreliable. Consider java.util.concurrent.ConcurrentLinkedQueue as an alternative. Investigation shows we are already using ConcurrentLinkedQueue in other locations so switch our uses of SynchronizedBuffer to ConcurrentLinkedQueue.
Author: Holden Karau <holden@us.ibm.com>
Closes#11059 from holdenk/SPARK-13164-replace-deprecated-synchronized-buffer-in-core.
Add a local property to indicate if checkpointing all RDDs that are marked with the checkpoint flag, and enable it in Streaming
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#10934 from zsxwing/recursive-checkpoint.
This is an existing issue uncovered recently by #10835. The reason for the exception was because the `SQLHistoryListener` gets all sorts of accumulators, not just the ones that represent SQL metrics. For example, the listener gets the `internal.metrics.shuffleRead.remoteBlocksFetched`, which is an Int, then it proceeds to cast the Int to a Long, which fails.
The fix is to mark accumulators representing SQL metrics using some internal metadata. Then we can identify which ones are SQL metrics and only process those in the `SQLHistoryListener`.
Author: Andrew Or <andrew@databricks.com>
Closes#10971 from andrewor14/fix-sql-history.
[SPARK-10873] Support column sort and search for History Server using jQuery DataTable and REST API. Before this commit, the history server was generated hard-coded html and can not support search, also, the sorting was disabled if there is any application that has more than one attempt. Supporting search and sort (over all applications rather than the 20 entries in the current page) in any case will greatly improve user experience.
1. Create the historypage-template.html for displaying application information in datables.
2. historypage.js uses jQuery to access the data from /api/v1/applications REST API, and use DataTable to display each application's information. For application that has more than one attempt, the RowsGroup is used to merge such entries while at the same time supporting sort and search.
3. "duration" and "lastUpdated" rest API are added to application's "attempts".
4. External javascirpt and css files for datatables, RowsGroup and jquery plugins are added with licenses clarified.
Snapshots for how it looks like now:
History page view:
![historypage](https://cloud.githubusercontent.com/assets/11683054/12184383/89bad774-b55a-11e5-84e4-b0276172976f.png)
Search:
![search](https://cloud.githubusercontent.com/assets/11683054/12184385/8d3b94b0-b55a-11e5-869a-cc0ef0a4242a.png)
Sort by started time:
![sort-by-started-time](https://cloud.githubusercontent.com/assets/11683054/12184387/8f757c3c-b55a-11e5-98c8-577936366566.png)
Author: zhuol <zhuol@yahoo-inc.com>
Closes#10648 from zhuoliu/10873.
Spark's `Partition` and `RDD.partitions` APIs have a contract which requires custom implementations of `RDD.partitions` to ensure that for all `x`, `rdd.partitions(x).index == x`; in other words, the `index` reported by a repartition needs to match its position in the partitions array.
If a custom RDD implementation violates this contract, then Spark has the potential to become stuck in an infinite recomputation loop when recomputing a subset of an RDD's partitions, since the tasks that are actually run will not correspond to the missing output partitions that triggered the recomputation. Here's a link to a notebook which demonstrates this problem: 5e8a5aa8d2/Violating%2520RDD.partitions%2520contract.html
In order to guard against this infinite loop behavior, this patch modifies Spark so that it fails fast and refuses to compute RDDs' whose `partitions` violate the API contract.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#10932 from JoshRosen/SPARK-13021.
The high level idea is that instead of having the executors send both accumulator updates and TaskMetrics, we should have them send only accumulator updates. This eliminates the need to maintain both code paths since one can be implemented in terms of the other. This effort is split into two parts:
**SPARK-12895: Implement TaskMetrics using accumulators.** TaskMetrics is basically just a bunch of accumulable fields. This patch makes TaskMetrics a syntactic wrapper around a collection of accumulators so we don't need to send TaskMetrics from the executors to the driver.
**SPARK-12896: Send only accumulator updates to the driver.** Now that TaskMetrics are expressed in terms of accumulators, we can capture all TaskMetrics values if we just send accumulator updates from the executors to the driver. This completes the parent issue SPARK-10620.
While an effort has been made to preserve as much of the public API as possible, there were a few known breaking DeveloperApi changes that would be very awkward to maintain. I will gather the full list shortly and post it here.
Note: This was once part of #10717. This patch is split out into its own patch from there to make it easier for others to review. Other smaller pieces of already been merged into master.
Author: Andrew Or <andrew@databricks.com>
Closes#10835 from andrewor14/task-metrics-use-accums.
Fix Java function API methods for flatMap and mapPartitions to require producing only an Iterator, not Iterable. Also fix DStream.flatMap to require a function producing TraversableOnce only, not Traversable.
CC rxin pwendell for API change; tdas since it also touches streaming.
Author: Sean Owen <sowen@cloudera.com>
Closes#10413 from srowen/SPARK-3369.
Added color coding to the Executors page for Active Tasks, Failed Tasks, Completed Tasks and Task Time.
Active Tasks is shaded blue with it's range based on percentage of total cores used.
Failed Tasks is shaded red ranging over the first 10% of total tasks failed
Completed Tasks is shaded green ranging over 10% of total tasks including failed and active tasks, but only when there are active or failed tasks on that executor.
Task Time is shaded red when GC Time goes over 10% of total time with it's range directly corresponding to the percent of total time.
Author: Alex Bozarth <ajbozart@us.ibm.com>
Closes#10154 from ajbozarth/spark12149.
- Remove Akka dependency from core. Note: the streaming-akka project still uses Akka.
- Remove HttpFileServer
- Remove Akka configs from SparkConf and SSLOptions
- Rename `spark.akka.frameSize` to `spark.rpc.message.maxSize`. I think it's still worth to keep this config because using `DirectTaskResult` or `IndirectTaskResult` depends on it.
- Update comments and docs
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#10854 from zsxwing/remove-akka.
This is a step in implementing SPARK-10620, which migrates TaskMetrics to accumulators.
TaskMetrics has a bunch of var's, some are fully public, some are `private[spark]`. This is bad coding style that makes it easy to accidentally overwrite previously set metrics. This has happened a few times in the past and caused bugs that were difficult to debug.
Instead, we should have get-or-create semantics, which are more readily understandable. This makes sense in the case of TaskMetrics because these are just aggregated metrics that we want to collect throughout the task, so it doesn't matter who's incrementing them.
Parent PR: #10717
Author: Andrew Or <andrew@databricks.com>
Author: Josh Rosen <joshrosen@databricks.com>
Author: andrewor14 <andrew@databricks.com>
Closes#10815 from andrewor14/get-or-create-metrics.
This is a small step in implementing SPARK-10620, which migrates TaskMetrics to accumulators. This patch is strictly a cleanup patch and introduces no change in functionality. It literally just renames 3 fields for consistency. Today we have:
```
inputMetrics.recordsRead
outputMetrics.bytesWritten
shuffleReadMetrics.localBlocksFetched
...
shuffleWriteMetrics.shuffleRecordsWritten
shuffleWriteMetrics.shuffleBytesWritten
shuffleWriteMetrics.shuffleWriteTime
```
The shuffle write ones are kind of redundant. We can drop the `shuffle` part in the method names. I added backward compatible (but deprecated) methods with the old names.
Parent PR: #10717
Author: Andrew Or <andrew@databricks.com>
Closes#10811 from andrewor14/rename-things.
This patch refactors portions of the BlockManager and CacheManager in order to avoid having to pass `evictedBlocks` lists throughout the code. It appears that these lists were only consumed by `TaskContext.taskMetrics`, so the new code now directly updates the metrics from the lower-level BlockManager methods.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#10776 from JoshRosen/SPARK-10985.
This pull request removes the external block store API. This is rarely used, and the file system interface is actually a better, more standard way to interact with external storage systems.
There are some other things to remove also, as pointed out by JoshRosen. We will do those as follow-up pull requests.
Author: Reynold Xin <rxin@databricks.com>
Closes#10752 from rxin/remove-offheap.
This patch significantly speeds up the BlockManagerSuite's "SPARK-9591: getRemoteBytes from another location when Exception throw" test, reducing the test time from 45s to ~250ms. The key change was to set `spark.shuffle.io.maxRetries` to 0 (the code previously set `spark.network.timeout` to `2s`, but this didn't make a difference because the slowdown was not due to this timeout).
Along the way, I also cleaned up the way that we handle SparkConf in BlockManagerSuite: previously, each test would mutate a shared SparkConf instance, while now each test gets a fresh SparkConf.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#10759 from JoshRosen/SPARK-12174.
When an Executor process is destroyed, the FileAppender that is asynchronously reading the stderr stream of the process can throw an IOException during read because the stream is closed. Before the ExecutorRunner destroys the process, the FileAppender thread is flagged to stop. This PR wraps the inputStream.read call of the FileAppender in a try/catch block so that if an IOException is thrown and the thread has been flagged to stop, it will safely ignore the exception. Additionally, the FileAppender thread was changed to use Utils.tryWithSafeFinally to better log any exception that do occur. Added unit tests to verify a IOException is thrown and logged if FileAppender is not flagged to stop, and that no IOException when the flag is set.
Author: Bryan Cutler <cutlerb@gmail.com>
Closes#10714 from BryanCutler/file-appender-read-ioexception-SPARK-9844.
We've already removed local execution but didn't deprecate `TaskContext.isRunningLocally()`; we should deprecate it for 2.0.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#10751 from JoshRosen/remove-local-exec-from-taskcontext.
This problem lies in `BypassMergeSortShuffleWriter`, empty partition will also generate a temp shuffle file with several bytes. So here change to only create file when partition is not empty.
This problem only lies in here, no such issue in `HashShuffleWriter`.
Please help to review, thanks a lot.
Author: jerryshao <sshao@hortonworks.com>
Closes#10376 from jerryshao/SPARK-12400.
Fix the style violation (space before , and :).
This PR is a followup for #10643
Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>
Closes#10719 from sarutak/SPARK-12692-followup-core.
[SPARK-12582][Test] IndexShuffleBlockResolverSuite fails in windows
* IndexShuffleBlockResolverSuite fails in windows due to file is not closed.
* mv IndexShuffleBlockResolverSuite.scala from "test/java" to "test/scala".
https://issues.apache.org/jira/browse/SPARK-12582
Author: Yucai Yu <yucai.yu@intel.com>
Closes#10526 from yucai/master.
This patch deduplicates some test code in BlockManagerSuite. I'm splitting this change off from a larger PR in order to make things easier to review.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#10667 from JoshRosen/block-mgr-tests-cleanup.
Replace Guava `Optional` with (an API clone of) Java 8 `java.util.Optional` (edit: and a clone of Guava `Optional`)
See also https://github.com/apache/spark/pull/10512
Author: Sean Owen <sowen@cloudera.com>
Closes#10513 from srowen/SPARK-4819.
Fix most build warnings: mostly deprecated API usages. I'll annotate some of the changes below. CC rxin who is leading the charge to remove the deprecated APIs.
Author: Sean Owen <sowen@cloudera.com>
Closes#10570 from srowen/SPARK-12618.
The default serializer in Kryo is FieldSerializer and it ignores transient fields and never calls `writeObject` or `readObject`. So we should register OpenHashMapBasedStateMap using `DefaultSerializer` to make it work with Kryo.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#10609 from zsxwing/SPARK-12591.
This PR removes `spark.cleaner.ttl` and the associated TTL-based metadata cleaning code.
Now that we have the `ContextCleaner` and a timer to trigger periodic GCs, I don't think that `spark.cleaner.ttl` is necessary anymore. The TTL-based cleaning isn't enabled by default, isn't included in our end-to-end tests, and has been a source of user confusion when it is misconfigured. If the TTL is set too low, data which is still being used may be evicted / deleted, leading to hard to diagnose bugs.
For all of these reasons, I think that we should remove this functionality in Spark 2.0. Additional benefits of doing this include marginally reduced memory usage, since we no longer need to store timetsamps in hashmaps, and a handful fewer threads.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#10534 from JoshRosen/remove-ttl-based-cleaning.
Change Java countByKey, countApproxDistinctByKey return types to use Java Long, not Scala; update similar methods for consistency on java.long.Long.valueOf with no API change
Author: Sean Owen <sowen@cloudera.com>
Closes#10554 from srowen/SPARK-12604.
Whole code of Vector.scala, VectorSuite.scala and GraphKryoRegistrator.scala are no longer used so it's time to remove them in Spark 2.0.
Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>
Closes#10613 from sarutak/SPARK-12665.
Cartesian product use UnsafeExternalSorter without comparator to do spilling, it will NPE if spilling happens.
This bug also hitted by #10605
cc JoshRosen
Author: Davies Liu <davies@databricks.com>
Closes#10606 from davies/fix_spilling.
I looked at each case individually and it looks like they can all be removed. The only one that I had to think twice was toArray (I even thought about un-deprecating it, until I realized it was a problem in Java to have toArray returning java.util.List).
Author: Reynold Xin <rxin@databricks.com>
Closes#10569 from rxin/SPARK-12615.
This patch updates the ExecutorRunner's terminate path to use the new java 8 API
to terminate processes more forcefully if possible. If the executor is unhealthy,
it would previously ignore the destroy() call. Presumably, the new java API was
added to handle cases like this.
We could update the termination path in the future to use OS specific commands
for older java versions.
Author: Nong Li <nong@databricks.com>
Closes#10438 from nongli/spark-12486-executors.
### Remove AkkaRpcEnv
Keep `SparkEnv.actorSystem` because Streaming still uses it. Will remove it and AkkaUtils after refactoring Streaming actorStream API.
### Remove systemName
There are 2 places using `systemName`:
* `RpcEnvConfig.name`. Actually, although it's used as `systemName` in `AkkaRpcEnv`, `NettyRpcEnv` uses it as the service name to output the log `Successfully started service *** on port ***`. Since the service name in log is useful, I keep `RpcEnvConfig.name`.
* `def setupEndpointRef(systemName: String, address: RpcAddress, endpointName: String)`. Each `ActorSystem` has a `systemName`. Akka requires `systemName` in its URI and will refuse a connection if `systemName` is not matched. However, `NettyRpcEnv` doesn't use it. So we can remove `systemName` from `setupEndpointRef` since we are removing `AkkaRpcEnv`.
### Remove RpcEnv.uriOf
`uriOf` exists because Akka uses different URI formats for with and without authentication, e.g., `akka.ssl.tcp...` and `akka.tcp://...`. But `NettyRpcEnv` uses the same format. So it's not necessary after removing `AkkaRpcEnv`.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#10459 from zsxwing/remove-akka-rpc-env.
We switched to TorrentBroadcast in Spark 1.1, and HttpBroadcast has been undocumented since then. It's time to remove it in Spark 2.0.
Author: Reynold Xin <rxin@databricks.com>
Closes#10531 from rxin/SPARK-12588.
The web UI's paginated table uses Javascript to implement certain navigation controls, such as table sorting and the "go to page" form. This is unnecessary and should be simplified to use plain HTML form controls and links.
/cc zsxwing, who wrote this original code, and yhuai.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#10441 from JoshRosen/simplify-paginated-table-sorting.
The feature was first added at commit: 7b877b2705 but was later removed (probably by mistake) at commit: fc8b58195a.
This change sets the default path of RDDs created via sc.textFile(...) to the path argument.
Here is the symptom:
* Using spark-1.5.2-bin-hadoop2.6:
scala> sc.textFile("/home/root/.bashrc").name
res5: String = null
scala> sc.binaryFiles("/home/root/.bashrc").name
res6: String = /home/root/.bashrc
* while using Spark 1.3.1:
scala> sc.textFile("/home/root/.bashrc").name
res0: String = /home/root/.bashrc
scala> sc.binaryFiles("/home/root/.bashrc").name
res1: String = /home/root/.bashrc
Author: Yaron Weinsberg <wyaron@gmail.com>
Author: yaron <yaron@il.ibm.com>
Closes#10456 from wyaron/master.
Restore the original value of os.arch property after each test
Since some of tests forced to set the specific value to os.arch property, we need to set the original value.
Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Closes#10289 from kiszk/SPARK-12311.
According the benchmark [1], LZ4-java could be 80% (or 30%) faster than Snappy.
After changing the compressor to LZ4, I saw 20% improvement on end-to-end time for a TPCDS query (Q4).
[1] https://github.com/ning/jvm-compressor-benchmark/wiki
cc rxin
Author: Davies Liu <davies@databricks.com>
Closes#10342 from davies/lz4.
When multiple workers exist in a host, we can bypass unnecessary remote access for broadcasts; block managers fetch broadcast blocks from the same host instead of remote hosts.
Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>
Closes#10346 from maropu/OptimizeBlockLocationOrder.
Added `channelActive` to `RpcHandler` so that `NettyRpcHandler` doesn't need `clients` any more.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#10301 from zsxwing/network-events.
In discussion (SPARK-9552), we proposed a force kill in `killExecutors`. But if there is nothing to kill, it will return back with true (acknowledgement). And then, it causes the certain executor(s) (which is not eligible to kill) adding to pendingToRemove list for further actions.
In this patch, we'd like to change the return semantics. If there is nothing to kill, we will return "false". and therefore all those non-eligible executors won't be added to the pendingToRemove list.
vanzin andrewor14 As the follow up of PR#7888, please let me know your comments.
Author: Grace <jie.huang@intel.com>
Author: Jie Huang <hjie@fosun.com>
Author: Andrew Or <andrew@databricks.com>
Closes#9796 from GraceH/emptyPendingToRemove.
Not jira is created.
The original test is passed because the class cast is lazy (only when the object's method is invoked).
Author: Jeff Zhang <zjffdu@apache.org>
Closes#10371 from zjffdu/minor_fix.
`DAGSchedulerEventLoop` normally only logs errors (so it can continue to process more events, from other jobs). However, this is not desirable in the tests -- the tests should be able to easily detect any exception, and also shouldn't silently succeed if there is an exception.
This was suggested by mateiz on https://github.com/apache/spark/pull/7699. It may have already turned up an issue in "zero split job".
Author: Imran Rashid <irashid@cloudera.com>
Closes#8466 from squito/SPARK-10248.
These changes rework the implementations of `SimpleFutureAction`, `ComplexFutureAction`, `JobWaiter`, and `AsyncRDDActions` such that asynchronous callbacks on the generated `Futures` NEVER block waiting for a job to complete. A small amount of mutex synchronization is necessary to protect the internal fields that manage cancellation, but these locks are only held very briefly and in practice should almost never cause any blocking to occur. The existing blocking APIs of these classes are retained, but they simply delegate to the underlying non-blocking API and `Await` the results with indefinite timeouts.
Associated JIRA ticket: https://issues.apache.org/jira/browse/SPARK-9026
Also fixes: https://issues.apache.org/jira/browse/SPARK-4514
This pull request contains all my own original work, which I release to the Spark project under its open source license.
Author: Richard W. Eggert II <richard.eggert@gmail.com>
Closes#9264 from reggert/fix-futureaction.
Fix a minor typo (unbalanced bracket) in ResetSystemProperties.
Author: Holden Karau <holden@us.ibm.com>
Closes#10303 from holdenk/SPARK-12332-trivial-typo-in-ResetSystemProperties-comment.
**Problem.** In unified memory management, acquiring execution memory may lead to eviction of storage memory. However, the space freed from evicting cached blocks is distributed among all active tasks. Thus, an incorrect upper bound on the execution memory per task can cause the acquisition to fail, leading to OOM's and premature spills.
**Example.** Suppose total memory is 1000B, cached blocks occupy 900B, `spark.memory.storageFraction` is 0.4, and there are two active tasks. In this case, the cap on task execution memory is 100B / 2 = 50B. If task A tries to acquire 200B, it will evict 100B of storage but can only acquire 50B because of the incorrect cap. For another example, see this [regression test](https://github.com/andrewor14/spark/blob/fix-oom/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala#L233) that I stole from JoshRosen.
**Solution.** Fix the cap on task execution memory. It should take into account the space that could have been freed by storage in addition to the current amount of memory available to execution. In the example above, the correct cap should have been 600B / 2 = 300B.
This patch also guards against the race condition (SPARK-12253):
(1) Existing tasks collectively occupy all execution memory
(2) New task comes in and blocks while existing tasks spill
(3) After tasks finish spilling, another task jumps in and puts in a large block, stealing the freed memory
(4) New task still cannot acquire memory and goes back to sleep
Author: Andrew Or <andrew@databricks.com>
Closes#10240 from andrewor14/fix-oom.
This patch adds documentation for Spark configurations that affect off-heap memory and makes some naming and validation improvements for those configs.
- Change `spark.memory.offHeapSize` to `spark.memory.offHeap.size`. This is fine because this configuration has not shipped in any Spark release yet (it's new in Spark 1.6).
- Deprecated `spark.unsafe.offHeap` in favor of a new `spark.memory.offHeap.enabled` configuration. The motivation behind this change is to gather all memory-related configurations under the same prefix.
- Add a check which prevents users from setting `spark.memory.offHeap.enabled=true` when `spark.memory.offHeap.size == 0`. After SPARK-11389 (#9344), which was committed in Spark 1.6, Spark enforces a hard limit on the amount of off-heap memory that it will allocate to tasks. As a result, enabling off-heap execution memory without setting `spark.memory.offHeap.size` will lead to immediate OOMs. The new configuration validation makes this scenario easier to diagnose, helping to avoid user confusion.
- Document these configurations on the configuration page.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#10237 from JoshRosen/SPARK-12251.
This avoids bringing up yet another HTTP server on the driver, and
instead reuses the file server already managed by the driver's
RpcEnv. As a bonus, the repl now inherits the security features of
the network library.
There's also a small change to create the directory for storing classes
under the root temp dir for the application (instead of directly
under java.io.tmpdir).
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#9923 from vanzin/SPARK-11563.
This patch fixes a bug in the eviction of storage memory by execution.
## The bug:
In general, execution should be able to evict storage memory when the total storage memory usage is greater than `maxMemory * spark.memory.storageFraction`. Due to a bug, however, Spark might wind up evicting no storage memory in certain cases where the storage memory usage was between `maxMemory * spark.memory.storageFraction` and `maxMemory`. For example, here is a regression test which illustrates the bug:
```scala
val maxMemory = 1000L
val taskAttemptId = 0L
val (mm, ms) = makeThings(maxMemory)
// Since we used the default storage fraction (0.5), we should be able to allocate 500 bytes
// of storage memory which are immune to eviction by execution memory pressure.
// Acquire enough storage memory to exceed the storage region size
assert(mm.acquireStorageMemory(dummyBlock, 750L, evictedBlocks))
assertEvictBlocksToFreeSpaceNotCalled(ms)
assert(mm.executionMemoryUsed === 0L)
assert(mm.storageMemoryUsed === 750L)
// At this point, storage is using 250 more bytes of memory than it is guaranteed, so execution
// should be able to reclaim up to 250 bytes of storage memory.
// Therefore, execution should now be able to require up to 500 bytes of memory:
assert(mm.acquireExecutionMemory(500L, taskAttemptId, MemoryMode.ON_HEAP) === 500L) // <--- fails by only returning 250L
assert(mm.storageMemoryUsed === 500L)
assert(mm.executionMemoryUsed === 500L)
assertEvictBlocksToFreeSpaceCalled(ms, 250L)
```
The problem relates to the control flow / interaction between `StorageMemoryPool.shrinkPoolToReclaimSpace()` and `MemoryStore.ensureFreeSpace()`. While trying to allocate the 500 bytes of execution memory, the `UnifiedMemoryManager` discovers that it will need to reclaim 250 bytes of memory from storage, so it calls `StorageMemoryPool.shrinkPoolToReclaimSpace(250L)`. This method, in turn, calls `MemoryStore.ensureFreeSpace(250L)`. However, `ensureFreeSpace()` first checks whether the requested space is less than `maxStorageMemory - storageMemoryUsed`, which will be true if there is any free execution memory because it turns out that `MemoryStore.maxStorageMemory = (maxMemory - onHeapExecutionMemoryPool.memoryUsed)` when the `UnifiedMemoryManager` is used.
The control flow here is somewhat confusing (it grew to be messy / confusing over time / as a result of the merging / refactoring of several components). In the pre-Spark 1.6 code, `ensureFreeSpace` was called directly by the `MemoryStore` itself, whereas in 1.6 it's involved in a confusing control flow where `MemoryStore` calls `MemoryManager.acquireStorageMemory`, which then calls back into `MemoryStore.ensureFreeSpace`, which, in turn, calls `MemoryManager.freeStorageMemory`.
## The solution:
The solution implemented in this patch is to remove the confusing circular control flow between `MemoryManager` and `MemoryStore`, making the storage memory acquisition process much more linear / straightforward. The key changes:
- Remove a layer of inheritance which made the memory manager code harder to understand (53841174760a24a0df3eb1562af1f33dbe340eb9).
- Move some bounds checks earlier in the call chain (13ba7ada77f87ef1ec362aec35c89a924e6987cb).
- Refactor `ensureFreeSpace()` so that the part which evicts blocks can be called independently from the part which checks whether there is enough free space to avoid eviction (7c68ca09cb1b12f157400866983f753ac863380e).
- Realize that this lets us remove a layer of overloads from `ensureFreeSpace` (eec4f6c87423d5e482b710e098486b3bbc4daf06).
- Realize that `ensureFreeSpace()` can simply be replaced with an `evictBlocksToFreeSpace()` method which is called [after we've already figured out](2dc842aea8/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala (L88)) how much memory needs to be reclaimed via eviction; (2dc842aea82c8895125d46a00aa43dfb0d121de9).
Along the way, I fixed some problems with the mocks in `MemoryManagerSuite`: the old mocks would [unconditionally](80a824d36e/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala (L84)) report that a block had been evicted even if there was enough space in the storage pool such that eviction would be avoided.
I also fixed a problem where `StorageMemoryPool._memoryUsed` might become negative due to freed memory being double-counted when excution evicts storage. The problem was that `StorageMemoryPoolshrinkPoolToFreeSpace` would [decrement `_memoryUsed`](7c68ca09cb (diff-935c68a9803be144ed7bafdd2f756a0fL133)) even though `StorageMemoryPool.freeMemory` had already decremented it as each evicted block was freed. See SPARK-12189 for details.
Author: Josh Rosen <joshrosen@databricks.com>
Author: Andrew Or <andrew@databricks.com>
Closes#10170 from JoshRosen/SPARK-12165.
Because of AM failure, the target executor number between driver and AM will be different, which will lead to unexpected behavior in dynamic allocation. So when AM is re-registered with driver, state in `ExecutorAllocationManager` and `CoarseGrainedSchedulerBacked` should be reset.
This issue is originally addressed in #8737 , here re-opened again. Thanks a lot KaiXinXiaoLei for finding this issue.
andrewor14 and vanzin would you please help to review this, thanks a lot.
Author: jerryshao <sshao@hortonworks.com>
Closes#9963 from jerryshao/SPARK-10582.
The json endpoint for stages doesn't include information on the stage duration that is present in the UI. This looks like a simple oversight, they should be included. eg., the metrics should be included at api/v1/applications/<appId>/stages.
Metrics I've added are: submissionTime, firstTaskLaunchedTime and completionTime
Author: Xin Ren <iamshrek@126.com>
Closes#10107 from keypointt/SPARK-11155.
`ByteBuffer` doesn't guarantee all contents in `ByteBuffer.array` are valid. E.g, a ByteBuffer returned by `ByteBuffer.slice`. We should not use the whole content of `ByteBuffer` unless we know that's correct.
This patch fixed all places that use `ByteBuffer.array` incorrectly.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#10083 from zsxwing/bytebuffer-array.
We should upgrade to SBT 0.13.9, since this is a requirement in order to use SBT's new Maven-style resolution features (which will be done in a separate patch, because it's blocked by some binary compatibility issues in the POM reader plugin).
I also upgraded Scalastyle to version 0.8.0, which was necessary in order to fix a Scala 2.10.5 compatibility issue (see https://github.com/scalastyle/scalastyle/issues/156). The newer Scalastyle is slightly stricter about whitespace surrounding tokens, so I fixed the new style violations.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#10112 from JoshRosen/upgrade-to-sbt-0.13.9.
This replaces https://github.com/apache/spark/pull/9696
Invoke Checkstyle and print any errors to the console, failing the step.
Use Google's style rules modified according to
https://cwiki.apache.org/confluence/display/SPARK/Spark+Code+Style+Guide
Some important checks are disabled (see TODOs in `checkstyle.xml`) due to
multiple violations being present in the codebase.
Suggest fixing those TODOs in a separate PR(s).
More on Checkstyle can be found on the [official website](http://checkstyle.sourceforge.net/).
Sample output (from [build 46345](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/46345/consoleFull)) (duplicated because I run the build twice with different profiles):
> Checkstyle checks failed at following occurrences:
[ERROR] src/main/java/org/apache/spark/sql/execution/datasources/parquet/UnsafeRowParquetRecordReader.java:[217,7] (coding) MissingSwitchDefault: switch without "default" clause.
> [ERROR] src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:[198,10] (modifier) ModifierOrder: 'protected' modifier out of order with the JLS suggestions.
> [ERROR] src/main/java/org/apache/spark/sql/execution/datasources/parquet/UnsafeRowParquetRecordReader.java:[217,7] (coding) MissingSwitchDefault: switch without "default" clause.
> [ERROR] src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:[198,10] (modifier) ModifierOrder: 'protected' modifier out of order with the JLS suggestions.
> [error] running /home/jenkins/workspace/SparkPullRequestBuilder2/dev/lint-java ; received return code 1
Also fix some of the minor violations that didn't require sweeping changes.
Apologies for the previous botched PRs - I finally figured out the issue.
cr: JoshRosen, pwendell
> I state that the contribution is my original work, and I license the work to the project under the project's open source license.
Author: Dmitry Erastov <derastov@gmail.com>
Closes#9867 from dskrvk/master.
**Problem.** Event logs in 1.6 were much bigger than 1.5. I ran page rank and the event log size in 1.6 was almost 5x that in 1.5. I did a bisect to find that the RDD callsite added in #9398 is largely responsible for this.
**Solution.** This patch removes the long form of the callsite (which is not used!) from the event log. This reduces the size of the event log significantly.
*Note on compatibility*: if this patch is to be merged into 1.6.0, then it won't break any compatibility. Otherwise, if it is merged into 1.6.1, then we might need to add more backward compatibility handling logic (currently does not exist yet).
Author: Andrew Or <andrew@databricks.com>
Closes#10115 from andrewor14/smaller-event-logs.
We should try increasing a timeout in NettyBlockTransferSecuritySuite in order to reduce that suite's flakiness in Jenkins.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#10113 from JoshRosen/SPARK-12082.
I have tried to address all the comments in pull request https://github.com/apache/spark/pull/2447.
Note that the second commit (using the new method in all internal code of all components) is quite intrusive and could be omitted.
Author: Jeroen Schot <jeroen.schot@surfsara.nl>
Closes#9767 from schot/master.
The existing `spark.memory.fraction` (default 0.75) gives the system 25% of the space to work with. For small heaps, this is not enough: e.g. default 1GB leaves only 250MB system memory. This is especially a problem in local mode, where the driver and executor are crammed in the same JVM. Members of the community have reported driver OOM's in such cases.
**New proposal.** We now reserve 300MB before taking the 75%. For 1GB JVMs, this leaves `(1024 - 300) * 0.75 = 543MB` for execution and storage. This is proposal (1) listed in the [JIRA](https://issues.apache.org/jira/browse/SPARK-12081).
Author: Andrew Or <andrew@databricks.com>
Closes#10081 from andrewor14/unified-memory-small-heaps.
The solution is the save the RDD partitioner in a separate file in the RDD checkpoint directory. That is, `<checkpoint dir>/_partitioner`. In most cases, whether the RDD partitioner was recovered or not, does not affect the correctness, only reduces performance. So this solution makes a best-effort attempt to save and recover the partitioner. If either fails, the checkpointing is not affected. This makes this patch safe and backward compatible.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#9983 from tdas/SPARK-12004.
This change seems large, but most of it is just replacing `byte[]`
with `ByteBuffer` and `new byte[]` with `ByteBuffer.allocate()`,
since it changes the network library's API.
The following are parts of the code that actually have meaningful
changes:
- The Message implementations were changed to inherit from a new
AbstractMessage that can optionally hold a reference to a body
(in the form of a ManagedBuffer); this is similar to how
ResponseWithBody worked before, except now it's not restricted
to just responses.
- The TransportFrameDecoder was pretty much rewritten to avoid
copies as much as possible; it doesn't rely on CompositeByteBuf
to accumulate incoming data anymore, since CompositeByteBuf
has issues when slices are retained. The code now is able to
create frames without having to resort to copying bytes except
for a few bytes (containing the frame length) in very rare cases.
- Some minor changes in the SASL layer to convert things back to
`byte[]` since the JDK SASL API operates on those.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#9987 from vanzin/SPARK-12007.
In the previous codes, `newDaemonCachedThreadPool` uses `SynchronousQueue`, which is wrong. `SynchronousQueue` is an empty queue that cannot cache any task. This patch uses `LinkedBlockingQueue` to fix it along with other fixes to make sure `newDaemonCachedThreadPool` can use at most `maxThreadNumber` threads, and after that, cache tasks to `LinkedBlockingQueue`.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#9978 from zsxwing/cached-threadpool.
This change does a couple of different things to make sure that the RpcEnv-level
code and the network library agree about the status of outstanding RPCs.
For RPCs that do not expect a reply ("RpcEnv.send"), support for one way
messages (hello CORBA!) was added to the network layer. This is a
"fire and forget" message that does not require any state to be kept
by the TransportClient; as a result, the RpcEnv 'Ack' message is not needed
anymore.
For RPCs that do expect a reply ("RpcEnv.ask"), the network library now
returns the internal RPC id; if the RpcEnv layer decides to time out the
RPC before the network layer does, it now asks the TransportClient to
forget about the RPC, so that if the network-level timeout occurs, the
client is not killed.
As part of implementing the above, I cleaned up some of the code in the
netty rpc backend, removing types that were not necessary and factoring
out some common code. Of interest is a slight change in the exceptions
when posting messages to a stopped RpcEnv; that's mostly to avoid nasty
error messages from the local-cluster backend when shutting down, which
pollutes the terminal output.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#9917 from vanzin/SPARK-11866.
- NettyRpcEnv::openStream() now correctly propagates errors to
the read side of the pipe.
- NettyStreamManager now throws if the file being transferred does
not exist.
- The network library now correctly handles zero-sized streams.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#9941 from vanzin/SPARK-11956.
This issue was addressed in https://github.com/apache/spark/pull/5494, but the fix in that PR, while safe in the sense that it will prevent the SparkContext from shutting down, misses the actual bug. The intent of `submitMissingTasks` should be understood as "submit the Tasks that are missing for the Stage, and run them as part of the ActiveJob identified by jobId". Because of a long-standing bug, the `jobId` parameter was never being used. Instead, we were trying to use the jobId with which the Stage was created -- which may no longer exist as an ActiveJob, hence the crash reported in SPARK-6880.
The correct fix is to use the ActiveJob specified by the supplied jobId parameter, which is guaranteed to exist at the call sites of submitMissingTasks.
This fix should be applied to all maintenance branches, since it has existed since 1.0.
kayousterhout pankajarora12
Author: Mark Hamstra <markhamstra@gmail.com>
Author: Imran Rashid <irashid@cloudera.com>
Closes#6291 from markhamstra/SPARK-6880.
This is continuation of SPARK-11761
Andrew suggested adding this protection. See tail of https://github.com/apache/spark/pull/9741
Author: tedyu <yuzhihong@gmail.com>
Closes#9852 from tedyu/master.
This change abstracts the code that serves jars / files to executors so that
each RpcEnv can have its own implementation; the akka version uses the existing
HTTP-based file serving mechanism, while the netty versions uses the new
stream support added to the network lib, which makes file transfers benefit
from the easier security configuration of the network library, and should also
reduce overhead overall.
The change includes a small fix to TransportChannelHandler so that it propagates
user events to downstream handlers.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#9530 from vanzin/SPARK-11140.
In PersistenceEngineSuite, we do not call `close()` on the PersistenceEngine at the end of the test. For the ZooKeeperPersistenceEngine, this causes us to leak a ZooKeeper client, causing the logs of unrelated tests to be periodically spammed with connection error messages from that client:
```
15/11/20 05:13:35.789 pool-1-thread-1-ScalaTest-running-PersistenceEngineSuite-SendThread(localhost:15741) INFO ClientCnxn: Opening socket connection to server localhost/127.0.0.1:15741. Will not attempt to authenticate using SASL (unknown error)
15/11/20 05:13:35.790 pool-1-thread-1-ScalaTest-running-PersistenceEngineSuite-SendThread(localhost:15741) WARN ClientCnxn: Session 0x15124ff48dd0000 for server null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: Connection refused
at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:739)
at org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:350)
at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1068)
```
This patch fixes this by using a `finally` block.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#9864 from JoshRosen/close-zookeeper-client-in-tests.
This patch reduces some RPC timeouts in order to speed up the slow "AkkaUtilsSuite.remote fetch ssl on - untrusted server", which used to take two minutes to run.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#9869 from JoshRosen/SPARK-11650.
To make sure that all lineage is correctly truncated for TrackStateRDD when checkpointed.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#9831 from tdas/SPARK-11845.
SparkListenerSuite's _"onTaskGettingResult() called when result fetched remotely"_ test was extremely slow (1 to 4 minutes to run) and recently became extremely flaky, frequently failing with OutOfMemoryError.
The root cause was the fact that this was using `System.setProperty` to set the Akka frame size, which was not actually modifying the frame size. As a result, this test would allocate much more data than necessary. The fix here is to simply use SparkConf in order to configure the frame size.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#9822 from JoshRosen/SPARK-11649.
[SPARK-6028](https://issues.apache.org/jira/browse/SPARK-6028) uses network module to implement RPC. However, there are some configurations named with `spark.shuffle` prefix in the network module.
This PR refactors them to make sure the user can control them in shuffle and RPC separately. The user can use `spark.rpc.*` to set the configuration for netty RPC.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#9481 from zsxwing/SPARK-10745.
https://issues.apache.org/jira/browse/SPARK-11792
The main changes include:
* Renaming `SizeEstimation` to `KnownSizeEstimation`. Hopefully this new name has more information.
* Making `estimatedSize` return `Long` instead of `Option[Long]`.
* In `UnsaveHashedRelation`, `estimatedSize` will delegate the work to `SizeEstimator` if we have not created a `BytesToBytesMap`.
Since we will put `UnsaveHashedRelation` to `BlockManager`, it is generally good to let it provide a more accurate size estimation. Also, if we do not put `BytesToBytesMap` directly into `BlockerManager`, I feel it is not really necessary to make `BytesToBytesMap` extends `KnownSizeEstimation`.
Author: Yin Huai <yhuai@databricks.com>
Closes#9813 from yhuai/SPARK-11792-followup.
Make sure we are using the context classloader when deserializing failed TaskResults instead of the Spark classloader.
The issue is that `enqueueFailedTask` was using the incorrect classloader which results in `ClassNotFoundException`.
Adds a test in TaskResultGetterSuite that compiles a custom exception, throws it on the executor, and asserts that Spark handles the TaskResult deserialization instead of returning `UnknownReason`.
See #9367 for previous comments
See SPARK-11195 for a full repro
Author: Hurshal Patel <hpatel516@gmail.com>
Closes#9779 from choochootrain/spark-11195-master.
This PR upgrade the version of RoaringBitmap to 0.5.10, to optimize the memory layout, will be much smaller when most of blocks are empty.
This PR is based on #9661 (fix conflicts), see all of the comments at https://github.com/apache/spark/pull/9661 .
Author: Kent Yao <yaooqinn@hotmail.com>
Author: Davies Liu <davies@databricks.com>
Author: Charles Allen <charles@allen-net.com>
Closes#9746 from davies/roaring_mapstatus.
By using the dynamic allocation, sometimes it occurs false killing for those busy executors. Some executors with assignments will be killed because of being idle for enough time (say 60 seconds). The root cause is that the Task-Launch listener event is asynchronized.
For example, some executors are under assigning tasks, but not sending out the listener notification yet. Meanwhile, the dynamic allocation's executor idle time is up (e.g., 60 seconds). It will trigger killExecutor event at the same time.
1. the timer expiration starts before the listener event arrives.
2. Then, the task is going to run on top of that killed/killing executor. It will lead to task failure finally.
Here is the proposal to fix it. We can add the force control for killExecutor. If the force control is not set (i.e., false), we'd better to check if the executor under killing is idle or busy. If the current executor has some assignment, we should not kill that executor and return back false (to indicate killing failure). In dynamic allocation, we'd better to turn off force killing (i.e., force = false), we will meet killing failure if tries to kill a busy executor. And then, the executor timer won't be invalid. Later on, the task assignment event arrives, we can remove the idle timer accordingly. So that we can avoid false killing for those busy executors in dynamic allocation.
For the rest of usages, the end users can decide if to use force killing or not by themselves. If to turn on that option, the killExecutor will do the action without any status checking.
Author: Grace <jie.huang@intel.com>
Author: Andrew Or <andrew@databricks.com>
Author: Jie Huang <jie.huang@intel.com>
Closes#7888 from GraceH/forcekill.
The basic idea is that:
The archive of the SparkR package itself, that is sparkr.zip, is created during build process and is contained in the Spark binary distribution. No change to it after the distribution is installed as the directory it resides ($SPARK_HOME/R/lib) may not be writable.
When there is R source code contained in jars or Spark packages specified with "--jars" or "--packages" command line option, a temporary directory is created by calling Utils.createTempDir() where the R packages built from the R source code will be installed. The temporary directory is writable, and won't interfere with each other when there are multiple SparkR sessions, and will be deleted when this SparkR session ends. The R binary packages installed in the temporary directory then are packed into an archive named rpkg.zip.
sparkr.zip and rpkg.zip are distributed to the cluster in YARN modes.
The distribution of rpkg.zip in Standalone modes is not supported in this PR, and will be address in another PR.
Various R files are updated to accept multiple lib paths (one is for SparkR package, the other is for other R packages) so that these package can be accessed in R.
Author: Sun Rui <rui.sun@intel.com>
Closes#9390 from sun-rui/SPARK-10500.
Currently, all the shuffle writer will write to target path directly, the file could be corrupted by other attempt of the same partition on the same executor. They should write to temporary file then rename to target path, as what we do in output committer. In order to make the rename atomic, the temporary file should be created in the same local directory (FileSystem).
This PR is based on #9214 , thanks to squito . Closes#9214
Author: Davies Liu <davies@databricks.com>
Closes#9610 from davies/safe_shuffle.
This patch aims to reduce the test time and flakiness of HiveSparkSubmitSuite, SparkSubmitSuite, and CliSuite.
Key changes:
- Disable IO synchronization calls for Derby writes, since durability doesn't matter for tests. This was done for HiveCompatibilitySuite in #6651 and resulted in huge test speedups.
- Add a few missing `--conf`s to disable various Spark UIs. The CliSuite, in particular, never disabled these UIs, leaving it prone to port-contention-related flakiness.
- Fix two instances where tests defined `beforeAll()` methods which were never called because the appropriate traits were not mixed in. I updated these tests suites to extend `BeforeAndAfterEach` so that they play nicely with our `ResetSystemProperties` trait.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#9623 from JoshRosen/SPARK-11647.
just trying to increase test coverage in the scheduler, this already works. It includes a regression test for SPARK-9809
copied some test utils from https://github.com/apache/spark/pull/5636, we can wait till that is merged first
Author: Imran Rashid <irashid@cloudera.com>
Closes#8402 from squito/test_retry_in_shared_shuffle_dep.
Changed AppClient to be non-blocking in `receiveAndReply` by using a separate thread to wait for response and reply to the context. The threads are managed by a thread pool. Also added unit tests for the AppClient interface.
Author: Bryan Cutler <bjcutler@us.ibm.com>
Closes#9317 from BryanCutler/appClient-receiveAndReply-SPARK-10827.
In order to lay the groundwork for proper off-heap memory support in SQL / Tungsten, we need to extend our MemoryManager to perform bookkeeping for off-heap memory.
## User-facing changes
This PR introduces a new configuration, `spark.memory.offHeapSize` (name subject to change), which specifies the absolute amount of off-heap memory that Spark and Spark SQL can use. If Tungsten is configured to use off-heap execution memory for allocating data pages, then all data page allocations must fit within this size limit.
## Internals changes
This PR contains a lot of internal refactoring of the MemoryManager. The key change at the heart of this patch is the introduction of a `MemoryPool` class (name subject to change) to manage the bookkeeping for a particular category of memory (storage, on-heap execution, and off-heap execution). These MemoryPools are not fixed-size; they can be dynamically grown and shrunk according to the MemoryManager's policies. In StaticMemoryManager, these pools have fixed sizes, proportional to the legacy `[storage|shuffle].memoryFraction`. In the new UnifiedMemoryManager, the sizes of these pools are dynamically adjusted according to its policies.
There are two subclasses of `MemoryPool`: `StorageMemoryPool` manages storage memory and `ExecutionMemoryPool` manages execution memory. The MemoryManager creates two execution pools, one for on-heap memory and one for off-heap. Instances of `ExecutionMemoryPool` manage the logic for fair sharing of their pooled memory across running tasks (in other words, the ShuffleMemoryManager-like logic has been moved out of MemoryManager and pushed into these ExecutionMemoryPool instances).
I think that this design is substantially easier to understand and reason about than the previous design, where most of these responsibilities were handled by MemoryManager and its subclasses. To see this, take at look at how simple the logic in `UnifiedMemoryManager` has become: it's now very easy to see when memory is dynamically shifted between storage and execution.
## TODOs
- [x] Fix handful of test failures in the MemoryManagerSuites.
- [x] Fix remaining TODO comments in code.
- [ ] Document new configuration.
- [x] Fix commented-out tests / asserts:
- [x] UnifiedMemoryManagerSuite.
- [x] Write tests that exercise the new off-heap memory management policies.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#9344 from JoshRosen/offheap-memory-accounting.
https://issues.apache.org/jira/browse/SPARK-10116
This is really trivial, just happened to notice it -- if `XORShiftRandom.hashSeed` is really supposed to have random bits throughout (as the comment implies), it needs to do something for the conversion to `long`.
mengxr mkolod
Author: Imran Rashid <irashid@cloudera.com>
Closes#8314 from squito/SPARK-10116.
This brings the support of off-heap memory for array inside BytesToBytesMap and InMemorySorter, then we could allocate all the memory from off-heap for execution.
Closes#8068
Author: Davies Liu <davies@databricks.com>
Closes#9477 from davies/unsafe_timsort.
OutputCommitCoordinator uses a map in a place where an array would suffice, increasing its memory consumption for result stages with millions of tasks.
This patch replaces that map with an array. The only tricky part of this is reasoning about the range of possible array indexes in order to make sure that we never index out of bounds.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#9274 from JoshRosen/SPARK-11307.
This is an updated version of #8995 by a-roberts. Original description follows:
Snappy now supports concatenation of serialized streams, this patch contains a version number change and the "does not support" test is now a "supports" test.
Snappy 1.1.2 changelog mentions:
> snappy-java-1.1.2 (22 September 2015)
> This is a backward compatible release for 1.1.x.
> Add AIX (32-bit) support.
> There is no upgrade for the native libraries of the other platforms.
> A major change since 1.1.1 is a support for reading concatenated results of SnappyOutputStream(s)
> snappy-java-1.1.2-RC2 (18 May 2015)
> Fix#107: SnappyOutputStream.close() is not idempotent
> snappy-java-1.1.2-RC1 (13 May 2015)
> SnappyInputStream now supports reading concatenated compressed results of SnappyOutputStream
> There has been no compressed format change since 1.0.5.x. So You can read the compressed results > interchangeablly between these versions.
> Fixes a problem when java.io.tmpdir does not exist.
Closes#8995.
Author: Adam Roberts <aroberts@uk.ibm.com>
Author: Josh Rosen <joshrosen@databricks.com>
Closes#9439 from JoshRosen/update-snappy.
In YARN mode, when preemption is enabled, we may leave executors in a
zombie state while we wait to retrieve the reason for which the executor
exited. This is so that we don't account for failed tasks that were
running on a preempted executor.
The issue is that while we wait for this information, the scheduler
might decide to schedule tasks on the executor, which will never be
able to run them. Other side effects include the block manager still
considering the executor available to cache blocks, for example.
So, when we know that an executor went down but we don't know why,
stop everything related to the executor, except its running tasks.
Only when we know the reason for the exit (or give up waiting for
it) we do update the running tasks.
This is achieved by a new `disableExecutor()` method in the
`Schedulable` interface. For managers that do not behave like this
(i.e. every one but YARN), the existing `executorLost()` method
will behave the same way it did before.
On top of that change, a few minor changes that made debugging easier,
and fixed some other minor issues:
- The cluster-mode AM was printing a misleading log message every
time an executor disconnected from the driver (because the akka
actor system was shared between driver and AM).
- Avoid sending unnecessary requests for an executor's exit reason
when we already know it was explicitly disabled / killed. This
avoids both multiple requests, and unnecessary requests that would
just cause warning messages on the AM (in the explicit kill case).
- Tone down a log message about the executor being lost when it
exited normally (e.g. preemption)
- Wake up the AM monitor thread when requests for executor loss
reasons arrive too, so that we can more quickly remove executors
from this zombie state.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#8887 from vanzin/SPARK-10622.
The test functionality should be the same, but without using mockito; logs don't
really say anything useful but I suspect it may be the cause of the flakiness,
since updating mocks when multiple threads may be using it doesn't work very
well. It also allows some other cleanup (= less test code in FsHistoryProvider).
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#9425 from vanzin/SPARK-11466.
DriverDescription refactored to case class because it included no mutable fields.
ApplicationDescription had one mutable field, which was appUiUrl. This field was set by the driver to point to the driver web UI. Master was modifying this field when the application was removed to redirect requests to history server. This was wrong because objects which are sent over the wire should be immutable. Now appUiUrl is immutable in ApplicationDescription and always points to the driver UI even if it is already shutdown. The UI url which master exposes to the user and modifies dynamically is now included into ApplicationInfo - a data object which describes the application state internally in master. That URL in ApplicationInfo is initialised with the value from ApplicationDescription.
ApplicationDescription also included value user, which is now a part of case class fields.
Author: Jacek Lewandowski <lewandowski.jacek@gmail.com>
Closes#9299 from jacek-lewandowski/SPARK-11344.
"Client mode" means the RPC env will not listen for incoming connections.
This allows certain processes in the Spark stack (such as Executors or
tha YARN client-mode AM) to act as pure clients when using the netty-based
RPC backend, reducing the number of sockets needed by the app and also the
number of open ports.
Client connections are also preferred when endpoints that actually have
a listening socket are involved; so, for example, if a Worker connects
to a Master and the Master needs to send a message to a Worker endpoint,
that client connection will be used, even though the Worker is also
listening for incoming connections.
With this change, the workaround for SPARK-10987 isn't necessary anymore, and
is removed. The AM connects to the driver in "client mode", and that connection
is used for all driver <-> AM communication, and so the AM is properly notified
when the connection goes down.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#9210 from vanzin/SPARK-10997.
JIRA: https://issues.apache.org/jira/browse/SPARK-11271
As reported in the JIRA ticket, when there are too many tasks, the memory usage of MapStatus will cause problem. Use BitSet instead of RoaringBitMap should be more efficient in memory usage.
Author: Liang-Chi Hsieh <viirya@appier.com>
Closes#9243 from viirya/mapstatus-bitset.
Use standard JDK APIs for that (with a little help from Guava). Most of the
changes here are in test code, since there were no tests specific to that
part of the code.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#9257 from vanzin/SPARK-11073.
Large HDFS clusters may take a while to leave safe mode when starting; this change
makes the HS wait for that before doing checks about its configuraton. This means
the HS won't stop right away if HDFS is in safe mode and the configuration is not
correct, but that should be a very uncommon situation.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#9043 from vanzin/SPARK-11020.
[SPARK-11338: HistoryPage not multi-tenancy enabled ...](https://issues.apache.org/jira/browse/SPARK-11338)
- `HistoryPage.scala` ...prepending all page links with the web proxy (`uiRoot`) path
- `HistoryServerSuite.scala` ...adding a test case to verify all site-relative links are prefixed when the environment variable `APPLICATION_WEB_PROXY_BASE` (or System property `spark.ui.proxyBase`) is set
Author: Christian Kadner <ckadner@us.ibm.com>
Closes#9291 from ckadner/SPARK-11338 and squashes the following commits:
01d2f35 [Christian Kadner] [SPARK-11338][WebUI] nit fixes
d054bd7 [Christian Kadner] [SPARK-11338][WebUI] prependBaseUri in method makePageLink
8bcb3dc [Christian Kadner] [SPARK-11338][WebUI] Prepend application links on HistoryPage with uiRoot path
Since we do not need to preserve a page before calling compute(), MapPartitionsWithPreparationRDD is not needed anymore.
This PR basically revert #8543, #8511, #8038, #8011
Author: Davies Liu <davies@databricks.com>
Closes#9381 from davies/remove_prepare2.
This PR introduce a mechanism to call spill() on those SQL operators that support spilling (for example, BytesToBytesMap, UnsafeExternalSorter and ShuffleExternalSorter) if there is not enough memory for execution. The preserved first page is needed anymore, so removed.
Other Spillable objects in Spark core (ExternalSorter and AppendOnlyMap) are not included in this PR, but those could benefit from this (trigger others' spilling).
The PrepareRDD may be not needed anymore, could be removed in follow up PR.
The following script will fail with OOM before this PR, finished in 150 seconds with 2G heap (also works in 1.5 branch, with similar duration).
```python
sqlContext.setConf("spark.sql.shuffle.partitions", "1")
df = sqlContext.range(1<<25).selectExpr("id", "repeat(id, 2) as s")
df2 = df.select(df.id.alias('id2'), df.s.alias('s2'))
j = df.join(df2, df.id==df2.id2).groupBy(df.id).max("id", "id2")
j.explain()
print j.count()
```
For thread-safety, here what I'm got:
1) Without calling spill(), the operators should only be used by single thread, no safety problems.
2) spill() could be triggered in two cases, triggered by itself, or by other operators. we can check trigger == this in spill(), so it's still in the same thread, so safety problems.
3) if it's triggered by other operators (right now cache will not trigger spill()), we only spill the data into disk when it's in scanning stage (building is finished), so the in-memory sorter or memory pages are read-only, we only need to synchronize the iterator and change it.
4) During scanning, the iterator will only use one record in one page, we can't free this page, because the downstream is currently using it (used by UnsafeRow or other objects). In BytesToBytesMap, we just skip the current page, and dump all others into disk. In UnsafeExternalSorter, we keep the page that is used by current record (having the same baseObject), free it when loading the next record. In ShuffleExternalSorter, the spill() will not trigger during scanning.
5) In order to avoid deadlock, we didn't call acquireMemory during spill (so we reused the pointer array in InMemorySorter).
Author: Davies Liu <davies@databricks.com>
Closes#9241 from davies/force_spill.
Commit af3bc59d1f introduced new
functionality so that if an executor dies for a reason that's not
caused by one of the tasks running on the executor (e.g., due to
pre-emption), Spark doesn't count the failure towards the maximum
number of failures for the task. That commit introduced some vague
naming that this commit attempts to fix; in particular:
(1) The variable "isNormalExit", which was used to refer to cases where
the executor died for a reason unrelated to the tasks running on the
machine, has been renamed (and reversed) to "exitCausedByApp". The problem
with the existing name is that it's not clear (at least to me!) what it
means for an exit to be "normal"; the new name is intended to make the
purpose of this variable more clear.
(2) The variable "shouldEventuallyFailJob" has been renamed to
"countTowardsTaskFailures". This variable is used to determine whether
a task's failure should be counted towards the maximum number of failures
allowed for a task before the associated Stage is aborted. The problem
with the existing name is that it can be confused with implying that
the task's failure should immediately cause the stage to fail because it
is somehow fatal (this is the case for a fetch failure, for example: if
a task fails because of a fetch failure, there's no point in retrying,
and the whole stage should be failed).
Author: Kay Ousterhout <kayousterhout@gmail.com>
Closes#9164 from kayousterhout/SPARK-11178.
… ReceiverTracker and ReceiverSchedulingPolicy to use it
This PR includes the following changes:
1. Add a new preferred location format, `executor_<host>_<executorID>` (e.g., "executor_localhost_2"), to support specifying the executor locations for RDD.
2. Use the new preferred location format in `ReceiverTracker` to optimize the starting time of Receivers when there are multiple executors in a host.
The goal of this PR is to enable the streaming scheduler to place receivers (which run as tasks) in specific executors. Basically, I want to have more control on the placement of the receivers such that they are evenly distributed among the executors. We tried to do this without changing the core scheduling logic. But it does not allow specifying particular executor as preferred location, only at the host level. So if there are two executors in the same host, and I want two receivers to run on them (one on each executor), I cannot specify that. Current code only specifies the host as preference, which may end up launching both receivers on the same executor. We try to work around it but restarting a receiver when it does not launch in the desired executor and hope that next time it will be started in the right one. But that cause lots of restarts, and delays in correctly launching the receiver.
So this change, would allow the streaming scheduler to specify the exact executor as the preferred location. Also this is not exposed to the user, only the streaming scheduler uses this.
Author: zsxwing <zsxwing@gmail.com>
Closes#9181 from zsxwing/executor-location.
This patch refactors the MemoryManager class structure. After #9000, Spark had the following classes:
- MemoryManager
- StaticMemoryManager
- ExecutorMemoryManager
- TaskMemoryManager
- ShuffleMemoryManager
This is fairly confusing. To simplify things, this patch consolidates several of these classes:
- ShuffleMemoryManager and ExecutorMemoryManager were merged into MemoryManager.
- TaskMemoryManager is moved into Spark Core.
**Key changes and tasks**:
- [x] Merge ExecutorMemoryManager into MemoryManager.
- [x] Move pooling logic into Allocator.
- [x] Move TaskMemoryManager from `spark-unsafe` to `spark-core`.
- [x] Refactor the existing Tungsten TaskMemoryManager interactions so Tungsten code use only this and not both this and ShuffleMemoryManager.
- [x] Refactor non-Tungsten code to use the TaskMemoryManager instead of ShuffleMemoryManager.
- [x] Merge ShuffleMemoryManager into MemoryManager.
- [x] Move code
- [x] ~~Simplify 1/n calculation.~~ **Will defer to followup, since this needs more work.**
- [x] Port ShuffleMemoryManagerSuite tests.
- [x] Move classes from `unsafe` package to `memory` package.
- [ ] Figure out how to handle the hacky use of the memory managers in HashedRelation's broadcast variable construction.
- [x] Test porting and cleanup: several tests relied on mock functionality (such as `TestShuffleMemoryManager.markAsOutOfMemory`) which has been changed or broken during the memory manager consolidation
- [x] AbstractBytesToBytesMapSuite
- [x] UnsafeExternalSorterSuite
- [x] UnsafeFixedWidthAggregationMapSuite
- [x] UnsafeKVExternalSorterSuite
**Compatiblity notes**:
- This patch introduces breaking changes in `ExternalAppendOnlyMap`, which is marked as `DevloperAPI` (likely for legacy reasons): this class now cannot be used outside of a task.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#9127 from JoshRosen/SPARK-10984.
This test can take a little while to finish on slow / loaded machines.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#9235 from vanzin/SPARK-11134.
```
// My machine only has 8 cores
$ bin/spark-shell --master local[32]
scala> val df = sc.parallelize(Seq((1, 1), (2, 2))).toDF("a", "b")
scala> df.as("x").join(df.as("y"), $"x.a" === $"y.a").count()
Caused by: java.io.IOException: Unable to acquire 2097152 bytes of memory
at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPage(UnsafeExternalSorter.java:351)
```
Author: Andrew Or <andrew@databricks.com>
Closes#9209 from andrewor14/fix-local-page-size.
There's a lot of duplication between SortShuffleManager and UnsafeShuffleManager. Given that these now provide the same set of functionality, now that UnsafeShuffleManager supports large records, I think that we should replace SortShuffleManager's serialized shuffle implementation with UnsafeShuffleManager's and should merge the two managers together.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#8829 from JoshRosen/consolidate-sort-shuffle-implementations.
Correct the logic to return `HDFSCacheTaskLocation` instance when the input `str` is a in memory location.
Author: zhichao.li <zhichao.li@intel.com>
Closes#9096 from zhichao-li/uselessBranch.
JIRA: https://issues.apache.org/jira/browse/SPARK-11051
When a `RDD` is materialized and checkpointed, its partitions and dependencies are cleared. If we allow local checkpointing on it and assign `LocalRDDCheckpointData` to its `checkpointData`. Next time when the RDD is materialized again, the error will be thrown.
Author: Liang-Chi Hsieh <viirya@appier.com>
Closes#9072 from viirya/no-localcheckpoint-after-checkpoint.
Because the registration RPC was not really an RPC, but a bunch of
disconnected messages, it was possible for other messages to be
sent before the reply to the registration arrived, and that would
confuse the Worker. Especially in local-cluster mode, the worker was
succeptible to receiving an executor request before it received a
message from the master saying registration succeeded.
On top of the above, the change also fixes a ClassCastException when
the registration fails, which also affects the executor registration
protocol. Because the `ask` is issued with a specific return type,
if the error message (of a different type) was returned instead, the
code would just die with an exception. This is fixed by having a common
base trait for these reply messages.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#9138 from vanzin/SPARK-11131.
#9084 uncovered that many tests that test spilling don't actually spill. This is a follow-up patch to fix that to ensure our unit tests actually catch potential bugs in spilling. The size of this patch is inflated by the refactoring of `ExternalSorterSuite`, which had a lot of duplicate code and logic.
Author: Andrew Or <andrew@databricks.com>
Closes#9124 from andrewor14/spilling-tests.
If the heartbeat receiver kills executors (and new ones are not registered to replace them), the idle timeout for the old executors will be lost (and then change a total number of executors requested by Driver), So new ones will be not to asked to replace them.
For example, executorsPendingToRemove=Set(1), and executor 2 is idle timeout before a new executor is asked to replace executor 1. Then driver kill executor 2, and sending RequestExecutors to AM. But executorsPendingToRemove=Set(1,2), So AM doesn't allocate a executor to replace 1.
see: https://github.com/apache/spark/pull/8668
Author: KaiXinXiaoLei <huleilei1@huawei.com>
Author: huleilei <huleilei1@huawei.com>
Closes#8945 from KaiXinXiaoLei/pendingexecutor.
Internal accumulators don't write the internal flag to event log. So on the history server Web UI, all accumulators are not internal. This causes incorrect peak execution memory and unwanted accumulator table displayed on the stage page.
To fix it, I add the "internal" property of AccumulableInfo when writing the event log.
Author: Carson Wang <carson.wang@intel.com>
Closes#9061 from carsonwang/accumulableBug.
Restrict tasks (of job) to only 1 to ensure that the causing Exception asserted for job failure is the deliberately thrown DAGSchedulerSuiteDummyException intended, not an UnsupportedOperationException from any second/subsequent tasks that can propagate from a race condition during code execution.
Author: shellberg <sah@zepler.org>
Closes#9076 from shellberg/shellberg-DAGSchedulerSuite-misbehavedResultHandlerTest-patch-1.
A few more changes:
1. Renamed IDVerifier -> RpcEndpointVerifier
2. Renamed NettyRpcAddress -> RpcEndpointAddress
3. Simplified NettyRpcHandler a bit by removing the connection count tracking. This is OK because I now force spark.shuffle.io.numConnectionsPerPeer to 1
4. Reduced spark.rpc.connect.threads to 64. It would be great to eventually remove this extra thread pool.
5. Minor cleanup & documentation.
Author: Reynold Xin <rxin@databricks.com>
Closes#9112 from rxin/SPARK-11096.
This patch unifies the memory management of the storage and execution regions such that either side can borrow memory from each other. When memory pressure arises, storage will be evicted in favor of execution. To avoid regressions in cases where storage is crucial, we dynamically allocate a fraction of space for storage that execution cannot evict. Several configurations are introduced:
- **spark.memory.fraction (default 0.75)**: fraction of the heap space used for execution and storage. The lower this is, the more frequently spills and cached data eviction occur. The purpose of this config is to set aside memory for internal metadata, user data structures, and imprecise size estimation in the case of sparse, unusually large records.
- **spark.memory.storageFraction (default 0.5)**: size of the storage region within the space set aside by `spark.memory.fraction`. Cached data may only be evicted if total storage exceeds this region.
- **spark.memory.useLegacyMode (default false)**: whether to use the memory management that existed in Spark 1.5 and before. This is mainly for backward compatibility.
For a detailed description of the design, see [SPARK-10000](https://issues.apache.org/jira/browse/SPARK-10000). This patch builds on top of the `MemoryManager` interface introduced in #9000.
Author: Andrew Or <andrew@databricks.com>
Closes#9084 from andrewor14/unified-memory-manager.
I'm going through the implementation right now for post-doc review. Adding more comments and renaming things as I go through them.
I also want to write higher level documentation about how the whole thing works -- but those will come in other pull requests.
Author: Reynold Xin <rxin@databricks.com>
Closes#9091 from rxin/rpc-review.
https://issues.apache.org/jira/browse/SPARK-10858
The issue here is that in resolveURI we default to calling new File(path).getAbsoluteFile().toURI(). But if the path passed in already has a # in it then File(path) will think that is supposed to be part of the actual file path and not a fragment so it changes # to %23. Then when we try to parse that later in Client as a URI it doesn't recognize there is a fragment.
so to fix we just check if there is a fragment, still create the File like we did before and then add the fragment back on.
Author: Tom Graves <tgraves@yahoo-inc.com>
Closes#9035 from tgravescs/SPARK-10858.
This change adds an API that encapsulates information about an app
launched using the library. It also creates a socket-based communication
layer for apps that are launched as child processes; the launching
application listens for connections from launched apps, and once
communication is established, the channel can be used to send updates
to the launching app, or to send commands to the child app.
The change also includes hooks for local, standalone/client and yarn
masters.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#7052 from vanzin/SPARK-8673.
This patch introduces a `MemoryManager` that is the central arbiter of how much memory to grant to storage and execution. This patch is primarily concerned only with refactoring while preserving the existing behavior as much as possible.
This is the first step away from the existing rigid separation of storage and execution memory, which has several major drawbacks discussed on the [issue](https://issues.apache.org/jira/browse/SPARK-10956). It is the precursor of a series of patches that will attempt to address those drawbacks.
Author: Andrew Or <andrew@databricks.com>
Author: Josh Rosen <joshrosen@databricks.com>
Author: andrewor14 <andrew@databricks.com>
Closes#9000 from andrewor14/memory-manager.
This PR just reverted 02144d6745 to remerge #6457 and also included the commits in #8905.
Author: zsxwing <zsxwing@gmail.com>
Closes#8944 from zsxwing/SPARK-6028.
Compatibility between history server script and functionality
The history server has its argument parsing class in HistoryServerArguments. However, this doesn't get involved in the start-history-server.sh codepath where the $0 arg is assigned to spark.history.fs.logDirectory and all other arguments discarded (e.g --property-file.)
This stops the other options being usable from this script
Author: Joshi <rekhajoshm@gmail.com>
Author: Rekha Joshi <rekhajoshm@gmail.com>
Closes#8758 from rekhajoshm/SPARK-10317.
Fix the following issues in StandaloneDynamicAllocationSuite:
1. It should not assume master and workers start in order
2. It should not assume master and workers get ready at once
3. It should not assume the application is already registered with master after creating SparkContext
4. It should not access Master.app and idToApp which are not thread safe
The changes includes:
* Use `eventually` to wait until master and workers are ready to fix 1 and 2
* Use `eventually` to wait until the application is registered with master to fix 3
* Use `askWithRetry[MasterStateResponse](RequestMasterState)` to get the application info to fix 4
Author: zsxwing <zsxwing@gmail.com>
Closes#8914 from zsxwing/fix-StandaloneDynamicAllocationSuite.
This makes two changes:
- Allow reduce tasks to fetch multiple map output partitions -- this is a pretty small change to HashShuffleFetcher
- Move shuffle locality computation out of DAGScheduler and into ShuffledRDD / MapOutputTracker; this was needed because the code in DAGScheduler wouldn't work for RDDs that fetch multiple map output partitions from each reduce task
I also added an AdaptiveSchedulingSuite that creates RDDs depending on multiple map output partitions.
Author: Matei Zaharia <matei@databricks.com>
Closes#8844 from mateiz/spark-9852.
The DiskBlockObjectWriter constructor took a BlockId parameter but never used it. As part of some general cleanup in these interfaces, this patch refactors its constructor to eliminate this parameter.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#8871 from JoshRosen/disk-block-object-writer-blockid-cleanup.
The current shuffle code has an interface named ShuffleReader with only one implementation, HashShuffleReader. This naming is confusing, since the same read path code is used for both sort- and hash-based shuffle. This patch addresses this by renaming HashShuffleReader to BlockStoreShuffleReader.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#8825 from JoshRosen/shuffle-reader-cleanup.
The job group, and job descriptions information is passed through thread local properties, and get inherited by child threads. In case of spark streaming, the streaming jobs inherit these properties from the thread that called streamingContext.start(). This may not make sense.
1. Job group: This is mainly used for cancelling a group of jobs together. It does not make sense to cancel streaming jobs like this, as the effect will be unpredictable. And its not a valid usecase any way, to cancel a streaming context, call streamingContext.stop()
2. Job description: This is used to pass on nice text descriptions for jobs to show up in the UI. The job description of the thread that calls streamingContext.start() is not useful for all the streaming jobs, as it does not make sense for all of the streaming jobs to have the same description, and the description may or may not be related to streaming.
The solution in this PR is meant for the Spark master branch, where local properties are inherited by cloning the properties. The job group and job description in the thread that starts the streaming scheduler are explicitly removed, so that all the subsequent child threads does not inherit them. Also, the starting is done in a new child thread, so that setting the job group and description for streaming, does not change those properties in the thread that called streamingContext.start().
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#8781 from tdas/SPARK-10649.
Track pending tasks by partition ID instead of Task objects.
Before this change, failure & retry could result in a case where a stage got submitted before the map output from its dependencies get registered. This was due to an error in the condition for registering map outputs.
Author: hushan[胡珊] <hushan@xiaomi.com>
Author: Imran Rashid <irashid@cloudera.com>
Closes#7699 from squito/SPARK-5259.
I noticed only one block manager registered with master in an unsuccessful build (https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-SBT/AMPLAB_JENKINS_BUILD_PROFILE=hadoop2.2,label=spark-test/3534/)
```
15/09/16 13:02:30.981 pool-1-thread-1-ScalaTest-running-BroadcastSuite INFO SparkContext: Running Spark version 1.6.0-SNAPSHOT
...
15/09/16 13:02:38.133 sparkDriver-akka.actor.default-dispatcher-19 INFO BlockManagerMasterEndpoint: Registering block manager localhost:48196 with 530.3 MB RAM, BlockManagerId(0, localhost, 48196)
```
In addition, the first block manager needed 7+ seconds to start. But the test expected 2 block managers so it failed.
However, there was no exception in this log file. So I checked a successful build (https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-SBT/3536/AMPLAB_JENKINS_BUILD_PROFILE=hadoop2.2,label=spark-test/) and it needed 4-5 seconds to set up the local cluster:
```
15/09/16 18:11:27.738 sparkWorker1-akka.actor.default-dispatcher-5 INFO Worker: Running Spark version 1.6.0-SNAPSHOT
...
15/09/16 18:11:30.838 sparkDriver-akka.actor.default-dispatcher-20 INFO BlockManagerMasterEndpoint: Registering block manager localhost:54202 with 530.3 MB RAM, BlockManagerId(1, localhost, 54202)
15/09/16 18:11:32.112 sparkDriver-akka.actor.default-dispatcher-20 INFO BlockManagerMasterEndpoint: Registering block manager localhost:32955 with 530.3 MB RAM, BlockManagerId(0, localhost, 32955)
```
In this build, the first block manager needed only 3+ seconds to start.
Comparing these two builds, I guess it's possible that the local cluster in `BroadcastSuite` cannot be ready in 10 seconds if the Jenkins worker is busy. So I just increased the timeout to 60 seconds to see if this can fix the issue.
Author: zsxwing <zsxwing@gmail.com>
Closes#8813 from zsxwing/fix-BroadcastSuite.
It does not make much sense to set `spark.shuffle.spill` or `spark.sql.planner.externalSort` to false: I believe that these configurations were initially added as "escape hatches" to guard against bugs in the external operators, but these operators are now mature and well-tested. In addition, these configurations are not handled in a consistent way anymore: SQL's Tungsten codepath ignores these configurations and will continue to use spilling operators. Similarly, Spark Core's `tungsten-sort` shuffle manager does not respect `spark.shuffle.spill=false`.
This pull request removes these configurations, adds warnings at the appropriate places, and deletes a large amount of code which was only used in code paths that did not support spilling.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#8831 from JoshRosen/remove-ability-to-disable-spilling.
When speculative execution is enabled, consider a scenario where the authorized committer of a particular output partition fails during the OutputCommitter.commitTask() call. In this case, the OutputCommitCoordinator is supposed to release that committer's exclusive lock on committing once that task fails. However, due to a unit mismatch (we used task attempt number in one place and task attempt id in another) the lock will not be released, causing Spark to go into an infinite retry loop.
This bug was masked by the fact that the OutputCommitCoordinator does not have enough end-to-end tests (the current tests use many mocks). Other factors contributing to this bug are the fact that we have many similarly-named identifiers that have different semantics but the same data types (e.g. attemptNumber and taskAttemptId, with inconsistent variable naming which makes them difficult to distinguish).
This patch adds a regression test and fixes this bug by always using task attempt numbers throughout this code.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#8544 from JoshRosen/SPARK-10381.
*Note: this is for master branch only.* The fix for branch-1.5 is at #8721.
The query execution ID is currently passed from a thread to its children, which is not the intended behavior. This led to `IllegalArgumentException: spark.sql.execution.id is already set` when running queries in parallel, e.g.:
```
(1 to 100).par.foreach { _ =>
sc.parallelize(1 to 5).map { i => (i, i) }.toDF("a", "b").count()
}
```
The cause is `SparkContext`'s local properties are inherited by default. This patch adds a way to exclude keys we don't want to be inherited, and makes SQL go through that code path.
Author: Andrew Or <andrew@databricks.com>
Closes#8710 from andrewor14/concurrent-sql-executions.
This patch adds support for submitting map stages in a DAG individually so that we can make downstream decisions after seeing statistics about their output, as part of SPARK-9850. I also added more comments to many of the key classes in DAGScheduler. By itself, the patch is not super useful except maybe to switch between a shuffle and broadcast join, but with the other subtasks of SPARK-9850 we'll be able to do more interesting decisions.
The main entry point is SparkContext.submitMapStage, which lets you run a map stage and see stats about the map output sizes. Other stats could also be collected through accumulators. See AdaptiveSchedulingSuite for a short example.
Author: Matei Zaharia <matei@databricks.com>
Closes#8180 from mateiz/spark-9851.
This is a follow-up patch to #8723. I missed one case there.
Author: Andrew Or <andrew@databricks.com>
Closes#8727 from andrewor14/fix-threading-suite.
This is a followup to #8499 which adds a Scalastyle rule to mandate the use of SparkHadoopUtil's JobContext accessor methods and fixes the existing violations.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#8521 from JoshRosen/SPARK-10330-part2.
Fix a few Java API test style issues: unused generic types, exceptions, wrong assert argument order
Author: Sean Owen <sowen@cloudera.com>
Closes#8706 from srowen/SPARK-10547.
This commit ensures if an assertion fails within a thread, it will ultimately fail the test. Otherwise we end up potentially masking real bugs by not propagating assertion failures properly.
Author: Andrew Or <andrew@databricks.com>
Closes#8723 from andrewor14/fix-threading-suite.
ShuffleManager implementations are currently not given type information for
the key, value and combiner classes. Serialization of shuffle objects relies
on objects being JavaSerializable, with methods defined for reading/writing
the object or, alternatively, serialization via Kryo which uses reflection.
Serialization systems like Avro, Thrift and Protobuf generate classes with
zero argument constructors and explicit schema information
(e.g. IndexedRecords in Avro have get, put and getSchema methods).
By serializing the key, value and combiner class names in ShuffleDependency,
shuffle implementations will have access to schema information when
registerShuffle() is called.
Author: Matt Massie <massie@cs.berkeley.edu>
Closes#7403 from massie/shuffle-classtags.
This is a regression introduced in #4960, this commit fixes it and adds a test.
tnachen andrewor14 please review, this should be an easy one.
Author: Iulian Dragos <jaguarul@gmail.com>
Closes#8653 from dragos/issue/mesos/fine-grained-maxExecutorCores.
The architecture is that, in YARN mode, if the driver detects that an executor has disconnected, it asks the ApplicationMaster why the executor died. If the ApplicationMaster is aware that the executor died because of preemption, all tasks associated with that executor are not marked as failed. The executor
is still removed from the driver's list of available executors, however.
There's a few open questions:
1. Should standalone mode have a similar "get executor loss reason" as well? I localized this change as much as possible to affect only YARN, but there could be a valid case to differentiate executor losses in standalone mode as well.
2. I make a pretty strong assumption in YarnAllocator that getExecutorLossReason(executorId) will only be called once per executor id; I do this so that I can remove the metadata from the in-memory map to avoid object accumulation. It's not clear if I'm being overly zealous to save space, however.
cc vanzin specifically for review because it collided with some earlier YARN scheduling work.
cc JoshRosen because it's similar to output commit coordination we did in the past
cc andrewor14 for our discussion on how to get executor exit codes and loss reasons
Author: mcheah <mcheah@palantir.com>
Closes#8007 from mccheah/feature/preemption-handling.
We introduced the Netty network module for shuffle in Spark 1.2, and has turned it on by default for 3 releases. The old ConnectionManager is difficult to maintain. If we merge the patch now, by the time it is released, it would be 1 yr for which ConnectionManager is off by default. It's time to remove it.
Author: Reynold Xin <rxin@databricks.com>
Closes#8161 from rxin/SPARK-9767.
[SPARK-9591](https://issues.apache.org/jira/browse/SPARK-9591)
When we getting the broadcast variable, we can fetch the block form several location,but now when connecting the lost blockmanager(idle for enough time removed by driver when using dynamic resource allocate and so on) will cause task fail,and the worse case will cause the job fail.
Author: jeanlyn <jeanlyn92@gmail.com>
Closes#7927 from jeanlyn/catch_exception.
This contribution is my original work and I license the work to the project under the project's open source license.
Author: Pat Shields <yeoldefortran@gmail.com>
Closes#7979 from pashields/env-loading-on-driver.
This is pretty minor, just trying to improve the readability of `DAGSchedulerSuite`, I figure every bit helps. Before whenever I read this test, I never knew what "should work" and "should be ignored" really meant -- this adds some asserts & updates comments to make it more clear. Also some reformatting per a suggestion from markhamstra on https://github.com/apache/spark/pull/7699
Author: Imran Rashid <irashid@cloudera.com>
Closes#8434 from squito/SPARK-10247.
The ```Stage``` class now tracks whether there were a sufficient number of consecutive failures of that stage to trigger an abort.
To avoid an infinite loop of stage retries, we abort the job completely after 4 consecutive stage failures for one stage. We still allow more than 4 consecutive stage failures if there is an intervening successful attempt for the stage, so that in very long-lived applications, where a stage may get reused many times, we don't abort the job after failures that have been recovered from successfully.
I've added test cases to exercise the most obvious scenarios.
Author: Ilya Ganelin <ilya.ganelin@capitalone.com>
Closes#5636 from ilganeli/SPARK-5945.
SPARK-4223.
Currently we support setting view and modify acls but you have to specify a list of users. It would be nice to support * meaning all users have access.
Manual tests to verify that: "*" works for any user in:
a. Spark ui: view and kill stage. Done.
b. Spark history server. Done.
c. Yarn application killing. Done.
Author: zhuol <zhuol@yahoo-inc.com>
Closes#8398 from zhuoliu/4223.
In SMJ, the first ExternalSorter could consume all the memory before spilling, then the second can not even acquire the first page.
Before we have a better memory allocator, SMJ should call prepare() before call any compute() of it's children.
cc rxin JoshRosen
Author: Davies Liu <davies@databricks.com>
Closes#8511 from davies/smj_memory.
This change aims at speeding up the dev cycle a little bit, by making
sure that all tests behave the same w.r.t. where the code to be tested
is loaded from. Namely, that means that tests don't rely on the assembly
anymore, rather loading all needed classes from the build directories.
The main change is to make sure all build directories (classes and test-classes)
are added to the classpath of child processes when running tests.
YarnClusterSuite required some custom code since the executors are run
differently (i.e. not through the launcher library, like standalone and
Mesos do).
I also found a couple of tests that could leak a SparkContext on failure,
and added code to handle those.
With this patch, it's possible to run the following command from a clean
source directory and have all tests pass:
mvn -Pyarn -Phadoop-2.4 -Phive-thriftserver install
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#7629 from vanzin/SPARK-9284.
Replace `JavaConversions` implicits with `JavaConverters`
Most occurrences I've seen so far are necessary conversions; a few have been avoidable. None are in critical code as far as I see, yet.
Author: Sean Owen <sowen@cloudera.com>
Closes#8033 from srowen/SPARK-9613.
The peak execution memory metric was introduced in SPARK-8735. That was before Tungsten was enabled by default, so it assumed that `spark.sql.unsafe.enabled` must be explicitly set to true. The result is that the memory is not displayed by default.
Author: Andrew Or <andrew@databricks.com>
Closes#8345 from andrewor14/show-memory-default.
https://issues.apache.org/jira/browse/SPARK-9439
In general, Yarn apps should be robust to NodeManager restarts. However, if you run spark with the external shuffle service on, after a NM restart all shuffles fail, b/c the shuffle service has lost some state with info on each executor. (Note the shuffle data is perfectly fine on disk across a NM restart, the problem is we've lost the small bit of state that lets us *find* those files.)
The solution proposed here is that the external shuffle service can write out its state to leveldb (backed by a local file) every time an executor is added. When running with yarn, that file is in the NM's local dir. Whenever the service is started, it looks for that file, and if it exists, it reads the file and re-registers all executors there.
Nothing is changed in non-yarn modes with this patch. The service is not given a place to save the state to, so it operates the same as before. This should make it easy to update other cluster managers as well, by just supplying the right file & the equivalent of yarn's `initializeApplication` -- I'm not familiar enough with those modes to know how to do that.
Author: Imran Rashid <irashid@cloudera.com>
Closes#7943 from squito/leveldb_external_shuffle_service_NM_restart and squashes the following commits:
0d285d3 [Imran Rashid] review feedback
70951d6 [Imran Rashid] Merge branch 'master' into leveldb_external_shuffle_service_NM_restart
5c71c8c [Imran Rashid] save executor to db before registering; style
2499c8c [Imran Rashid] explicit dependency on jackson-annotations
795d28f [Imran Rashid] review feedback
81f80e2 [Imran Rashid] Merge branch 'master' into leveldb_external_shuffle_service_NM_restart
594d520 [Imran Rashid] use json to serialize application executor info
1a7980b [Imran Rashid] version
8267d2a [Imran Rashid] style
e9f99e8 [Imran Rashid] cleanup the handling of bad dbs a little
9378ba3 [Imran Rashid] fail gracefully on corrupt leveldb files
acedb62 [Imran Rashid] switch to writing out one record per executor
79922b7 [Imran Rashid] rely on yarn to call stopApplication; assorted cleanup
12b6a35 [Imran Rashid] save registered executors when apps are removed; add tests
c878fbe [Imran Rashid] better explanation of shuffle service port handling
694934c [Imran Rashid] only open leveldb connection once per service
d596410 [Imran Rashid] store executor data in leveldb
59800b7 [Imran Rashid] Files.move in case renaming is unsupported
32fe5ae [Imran Rashid] Merge branch 'master' into external_shuffle_service_NM_restart
d7450f0 [Imran Rashid] style
f729e2b [Imran Rashid] debugging
4492835 [Imran Rashid] lol, dont use a PrintWriter b/c of scalastyle checks
0a39b98 [Imran Rashid] Merge branch 'master' into external_shuffle_service_NM_restart
55f49fc [Imran Rashid] make sure the service doesnt die if the registered executor file is corrupt; add tests
245db19 [Imran Rashid] style
62586a6 [Imran Rashid] just serialize the whole executors map
bdbbf0d [Imran Rashid] comments, remove some unnecessary changes
857331a [Imran Rashid] better tests & comments
bb9d1e6 [Imran Rashid] formatting
bdc4b32 [Imran Rashid] rename
86e0cb9 [Imran Rashid] for tests, shuffle service finds an open port
23994ff [Imran Rashid] style
7504de8 [Imran Rashid] style
a36729c [Imran Rashid] cleanup
efb6195 [Imran Rashid] proper unit test, and no longer leak if apps stop during NM restart
dd93dc0 [Imran Rashid] test for shuffle service w/ NM restarts
d596969 [Imran Rashid] cleanup imports
0e9d69b [Imran Rashid] better names
9eae119 [Imran Rashid] cleanup lots of duplication
1136f44 [Imran Rashid] test needs to have an actual shuffle
0b588bd [Imran Rashid] more fixes ...
ad122ef [Imran Rashid] more fixes
5e5a7c3 [Imran Rashid] fix build
c69f46b [Imran Rashid] maybe working version, needs tests & cleanup ...
bb3ba49 [Imran Rashid] minor cleanup
36127d3 [Imran Rashid] wip
b9d2ced [Imran Rashid] incomplete setup for external shuffle service tests
In Scala, `Seq.fill` always seems to return a List. Accessing a list by index is an O(N) operation. Thus, the following code will be really slow (~10 seconds on my machine):
```scala
val numItems = 100000
val s = Seq.fill(numItems)(1)
for (i <- 0 until numItems) s(i)
```
It turns out that we had a loop like this in DAGScheduler code, although it's a little tricky to spot. In `getPreferredLocsInternal`, there's a call to `getCacheLocs(rdd)(partition)`. The `getCacheLocs` call returns a Seq. If this Seq is a List and the RDD contains many partitions, then indexing into this list will cost O(partitions). Thus, when we loop over our tasks to compute their individual preferred locations we implicitly perform an N^2 loop, reducing scheduling throughput.
This patch fixes this by replacing `Seq` with `Array`.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#8178 from JoshRosen/dagscheduler-perf.
The shuffle locality patch made the DAGScheduler aware of shuffle data,
but for RDDs that have both narrow and shuffle dependencies, it can
cause them to place tasks based on the shuffle dependency instead of the
narrow one. This case is common in iterative join-based algorithms like
PageRank and ALS, where one RDD is hash-partitioned and one isn't.
Author: Matei Zaharia <matei@databricks.com>
Closes#8220 from mateiz/shuffle-loc-fix.
In these tests, we use a custom listener and we assert on fields in the stage / task completion events. However, these events are posted in a separate thread so they're not guaranteed to be posted in time. This commit fixes this flakiness through a job end registration callback.
Author: Andrew Or <andrew@databricks.com>
Closes#8176 from andrewor14/fix-accumulator-suite.
I think that we should pass additional configuration flags to disable the driver UI and Master REST server in SparkSubmitSuite and HiveSparkSubmitSuite. This might cut down on port-contention-related flakiness in Jenkins.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#8124 from JoshRosen/disable-ui-in-sparksubmitsuite.
… allocation are set. Now, dynamic allocation is set to false when num-executors is explicitly specified as an argument. Consequently, executorAllocationManager in not initialized in the SparkContext.
Author: Niranjan Padmanabhan <niranjan.padmanabhan@cloudera.com>
Closes#7657 from neurons/SPARK-9092.
This is the sister patch to #8011, but for aggregation.
In a nutshell: create the `TungstenAggregationIterator` before computing the parent partition. Internally this creates a `BytesToBytesMap` which acquires a page in the constructor as of this patch. This ensures that the aggregation operator is not starved since we reserve at least 1 page in advance.
rxin yhuai
Author: Andrew Or <andrew@databricks.com>
Closes#8038 from andrewor14/unsafe-starve-memory-agg.
This is based on KaiXinXiaoLei's changes in #7716.
The issue is that when someone calls `sc.killExecutor("1")` on the same executor twice quickly, then the executor target will be adjusted downwards by 2 instead of 1 even though we're only actually killing one executor. In certain cases where we don't adjust the target back upwards quickly, we'll end up with jobs hanging.
This is a common danger because there are many places where this is called:
- `HeartbeatReceiver` kills an executor that has not been sending heartbeats
- `ExecutorAllocationManager` kills an executor that has been idle
- The user code might call this, which may interfere with the previous callers
While it's not clear whether this fixes SPARK-9745, fixing this potential race condition seems like a strict improvement. I've added a regression test to illustrate the issue.
Author: Andrew Or <andrew@databricks.com>
Closes#8078 from andrewor14/da-double-kill.
This allows clients to retrieve the original exception from the
cause field of the SparkException that is thrown by the driver.
If the original exception is not in fact Serializable then it will
not be returned, but the message and stacktrace will be. (All Java
Throwables implement the Serializable interface, but this is no
guarantee that a particular implementation can actually be
serialized.)
Author: Tom White <tom@cloudera.com>
Closes#7014 from tomwhite/propagate-user-exceptions.
`InternalAccumulator.create` doesn't call `registerAccumulatorForCleanup` to register itself with ContextCleaner, so `WeakReference`s for these accumulators in `Accumulators.originals` won't be removed.
This PR added `registerAccumulatorForCleanup` for internal accumulators to avoid the memory leak.
Author: zsxwing <zsxwing@gmail.com>
Closes#8108 from zsxwing/internal-accumulators-leak.
PlatformDependent.UNSAFE is way too verbose.
Author: Reynold Xin <rxin@databricks.com>
Closes#8094 from rxin/SPARK-9815 and squashes the following commits:
229b603 [Reynold Xin] [SPARK-9815] Rename PlatformDependent.UNSAFE -> Platform.
The issue only happens if `spark.executor.cores` is not set and executor memory is set to a high value.
For example, if we have a worker with 4G and 10 cores and we set `spark.executor.memory` to 3G, then only 1 core is assigned to the executor. The correct number should be 10 cores.
I've added a unit test to illustrate the issue.
Author: Carson Wang <carson.wang@intel.com>
Closes#8017 from carsonwang/SPARK-9731 and squashes the following commits:
d09ec48 [Carson Wang] Fix code style
86b651f [Carson Wang] Simplify the code
943cc4c [Carson Wang] fix scheduling correct cores to executors
The original code that this test tests is removed in 9270bd06fd. It was ignored shortly before that so we never caught it. This patch re-enables the test and adds the code necessary to make it pass.
JoshRosen yhuai
Author: Andrew Or <andrew@databricks.com>
Closes#8015 from andrewor14/SPARK-9674 and squashes the following commits:
225eac2 [Andrew Or] Merge branch 'master' of github.com:apache/spark into SPARK-9674
8c24209 [Andrew Or] Fix NPE
e541d64 [Andrew Or] Track aggregation memory for both sort and hash
0be3a42 [Andrew Or] Fix test
This patch follows exactly #7891 (except testing)
Author: Davies Liu <davies@databricks.com>
Closes#8005 from davies/larger_record and squashes the following commits:
f9c4aff [Davies Liu] address comments
9de5c72 [Davies Liu] support records larger than page size in UnsafeShuffleExternalSorter
Previously, we use 64MB as the default page size, which was way too big for a lot of Spark applications (especially for single node).
This patch changes it so that the default page size, if unset by the user, is determined by the number of cores available and the total execution memory available.
Author: Reynold Xin <rxin@databricks.com>
Closes#8012 from rxin/pagesize and squashes the following commits:
16f4756 [Reynold Xin] Fixed failing test.
5afd570 [Reynold Xin] private...
0d5fb98 [Reynold Xin] Update default value.
674a6cd [Reynold Xin] Address review feedback.
dc00e05 [Reynold Xin] Merge with master.
73ebdb6 [Reynold Xin] [SPARK-9700] Pick default page size more intelligently.
The issue is that a task may run multiple sorts, and the sorts run by the child operator (i.e. parent RDD) may acquire all available memory such that other sorts in the same task do not have enough to proceed. This manifests itself in an `IOException("Unable to acquire X bytes of memory")` thrown by `UnsafeExternalSorter`.
The solution is to reserve a page in each sorter in the chain before computing the child operator's (parent RDD's) partitions. This requires us to use a new special RDD that does some preparation before computing the parent's partitions.
Author: Andrew Or <andrew@databricks.com>
Closes#8011 from andrewor14/unsafe-starve-memory and squashes the following commits:
35b69a4 [Andrew Or] Simplify test
0b07782 [Andrew Or] Minor: update comments
5d5afdf [Andrew Or] Merge branch 'master' of github.com:apache/spark into unsafe-starve-memory
254032e [Andrew Or] Add tests
234acbd [Andrew Or] Reserve a page in sorter when preparing each partition
b889e08 [Andrew Or] MapPartitionsWithPreparationRDD
This pull request adds a destructive iterator to BytesToBytesMap. When used, the iterator frees pages as it traverses them. This is part of the effort to avoid starving when we have more than one operators that can exhaust memory.
This is based on #7924, but fixes a bug there (Don't use destructive iterator in UnsafeKVExternalSorter).
Closes#7924.
Author: Liang-Chi Hsieh <viirya@appier.com>
Author: Reynold Xin <rxin@databricks.com>
Closes#8003 from rxin/map-destructive-iterator and squashes the following commits:
6b618c3 [Reynold Xin] Don't use destructive iterator in UnsafeKVExternalSorter.
a7bd8ec [Reynold Xin] Merge remote-tracking branch 'viirya/destructive_iter' into map-destructive-iterator
7652083 [Liang-Chi Hsieh] For comments: add destructiveIterator(), modify unit test, remove code block.
4a3e9de [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into destructive_iter
581e9e3 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into destructive_iter
f0ff783 [Liang-Chi Hsieh] No need to free last page.
9e9d2a3 [Liang-Chi Hsieh] Add a destructive iterator for BytesToBytesMap.
First, it's probably a bad idea to call generated Scala methods
from Java. In this case, the method being called wasn't actually
"Utils.createTempDir()", but actually the method that returns the
first default argument to the actual createTempDir method, which
is just the location of java.io.tmpdir; meaning that all tests in
the class were using the same temp dir, and thus affecting each
other.
Second, spillingOccursInResponseToMemoryPressure was not writing
enough records to actually cause a spill.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#7970 from vanzin/SPARK-9651 and squashes the following commits:
74d357f [Marcelo Vanzin] Clean up temp dir on test tear down.
a64f36a [Marcelo Vanzin] [SPARK-9651] Fix UnsafeExternalSorterSuite.
```
Error Message
Failed to bind to: /127.0.0.1:7093: Service 'sparkMaster' failed after 16 retries!
Stacktrace
java.net.BindException: Failed to bind to: /127.0.0.1:7093: Service 'sparkMaster' failed after 16 retries!
at org.jboss.netty.bootstrap.ServerBootstrap.bind(ServerBootstrap.java:272)
at akka.remote.transport.netty.NettyTransport$$anonfun$listen$1.apply(NettyTransport.scala:393)
at akka.remote.transport.netty.NettyTransport$$anonfun$listen$1.apply(NettyTransport.scala:389)
at scala.util.Success$$anonfun$map$1.apply(Try.scala:206)
at scala.util.Try$.apply(Try.scala:161)
```
Author: Andrew Or <andrew@databricks.com>
Closes#7968 from andrewor14/fix-master-flaky-test and squashes the following commits:
fcc42ef [Andrew Or] Randomize port
The current implementation of UnsafeExternalSort uses NoOpPrefixComparator for binary-typed data.
So, we need to add BinaryPrefixComparator in PrefixComparators.
Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>
Closes#7676 from maropu/BinaryTypePrefixComparator and squashes the following commits:
fe6f31b [Takeshi YAMAMURO] Apply comments
d943c04 [Takeshi YAMAMURO] Add a codegen'd entry for BinaryType in SortPrefix
ecf3ac5 [Takeshi YAMAMURO] Support BinaryType in PrefixComparator
shivaram cafreeman Could you please help me in testing this out? Exposing and running `rPackageBuilder` from inside the shell works, but for some reason, I can't get it to work during Spark Submit. It just starts relaunching Spark Submit.
For testing, you may use the R branch with [sbt-spark-package](https://github.com/databricks/sbt-spark-package). You can call spPackage, and then pass the jar using `--jars`.
Author: Burak Yavuz <brkyvz@gmail.com>
Closes#7139 from brkyvz/r-submit and squashes the following commits:
0de384f [Burak Yavuz] remove unused imports 2
d253708 [Burak Yavuz] removed unused imports
6603d0d [Burak Yavuz] addressed comments
4258ffe [Burak Yavuz] merged master
ddfcc06 [Burak Yavuz] added zipping test
3a1be7d [Burak Yavuz] don't zip
77995df [Burak Yavuz] fix URI
ac45527 [Burak Yavuz] added zipping of all libs
e6bf7b0 [Burak Yavuz] add println ignores
1bc5554 [Burak Yavuz] add assumes for tests
9778e03 [Burak Yavuz] addressed comments
b42b300 [Burak Yavuz] merged master
ffd134e [Burak Yavuz] Merge branch 'master' of github.com:apache/spark into r-submit
d867756 [Burak Yavuz] add apache header
eff5ba1 [Burak Yavuz] ready for review
8838edb [Burak Yavuz] Merge branch 'master' of github.com:apache/spark into r-submit
e5b5a06 [Burak Yavuz] added doc
bb751ce [Burak Yavuz] fix null bug
0226768 [Burak Yavuz] fixed issues
8810beb [Burak Yavuz] R packages support
https://issues.apache.org/jira/browse/SPARK-9602
Although we have hidden Akka behind RPC interface, I found that the Akka/Actor-related comments are still spreading everywhere. To make it consistent, we shall remove "actor"/"akka" words from the comments...
Author: CodingCat <zhunansjtu@gmail.com>
Closes#7936 from CodingCat/SPARK-9602 and squashes the following commits:
e8296a3 [CodingCat] remove actor words from comments
This patch extends UnsafeExternalSorter to support records larger than the page size. The basic strategy is the same as in #7762: store large records in their own overflow pages.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#7891 from JoshRosen/large-records-in-sql-sorter and squashes the following commits:
967580b [Josh Rosen] Merge remote-tracking branch 'origin/master' into large-records-in-sql-sorter
948c344 [Josh Rosen] Add large records tests for KV sorter.
3c17288 [Josh Rosen] Combine memory and disk cleanup into general cleanupResources() method
380f217 [Josh Rosen] Merge remote-tracking branch 'origin/master' into large-records-in-sql-sorter
27eafa0 [Josh Rosen] Fix page size in PackedRecordPointerSuite
a49baef [Josh Rosen] Address initial round of review comments
3edb931 [Josh Rosen] Remove accidentally-committed debug statements.
2b164e2 [Josh Rosen] Support large records in UnsafeExternalSorter.
Enable most javac lint warnings; fix a lot of build warnings. In a few cases, touch up surrounding code in the process.
I'll explain several of the changes inline in comments.
Author: Sean Owen <sowen@cloudera.com>
Closes#7862 from srowen/SPARK-9534 and squashes the following commits:
ea51618 [Sean Owen] Enable most javac lint warnings; fix a lot of build warnings. In a few cases, touch up surrounding code in the process.
While the functionality is there to exclude packages, there are no flags that allow users to exclude dependencies, in case of dependency conflicts. We should provide users with a flag to add dependency exclusions in case the packages are not resolved properly (or not available due to licensing).
The flag I added was --packages-exclude, but I'm open on renaming it. I also added property flags in case people would like to use a conf file to provide dependencies, which is possible if there is a long list of dependencies or exclusions.
cc andrewor14 vanzin pwendell
Author: Burak Yavuz <brkyvz@gmail.com>
Closes#7599 from brkyvz/packages-exclusions and squashes the following commits:
636f410 [Burak Yavuz] addressed nits
6e54ede [Burak Yavuz] is this the culprit
b5e508e [Burak Yavuz] Merge branch 'master' of github.com:apache/spark into packages-exclusions
154f5db [Burak Yavuz] addressed initial comments
1536d7a [Burak Yavuz] Added flags to exclude packages using --packages-exclude
Certain use cases of Spark involve RDDs with long lineages that must be truncated periodically (e.g. GraphX). The existing way of doing it is through `rdd.checkpoint()`, which is expensive because it writes to HDFS. This patch provides an alternative to truncate lineages cheaply *without providing the same level of fault tolerance*.
**Local checkpointing** writes checkpointed data to the local file system through the block manager. It is much faster than replicating to a reliable storage and provides the same semantics as long as executors do not fail. It is accessible through a new operator `rdd.localCheckpoint()` and leaves the old one unchanged. Users may even decide to combine the two and call the reliable one less frequently.
The bulk of this patch involves refactoring the checkpointing interface to accept custom implementations of checkpointing. [Design doc](https://issues.apache.org/jira/secure/attachment/12741708/SPARK-7292-design.pdf).
Author: Andrew Or <andrew@databricks.com>
Closes#7279 from andrewor14/local-checkpoint and squashes the following commits:
729600f [Andrew Or] Oops, fix tests
34bc059 [Andrew Or] Avoid computing all partitions in local checkpoint
e43bbb6 [Andrew Or] Merge branch 'master' of github.com:apache/spark into local-checkpoint
3be5aea [Andrew Or] Address comments
bf846a6 [Andrew Or] Merge branch 'master' of github.com:apache/spark into local-checkpoint
ab003a3 [Andrew Or] Fix compile
c2e111b [Andrew Or] Address comments
33f167a [Andrew Or] Merge branch 'master' of github.com:apache/spark into local-checkpoint
e908a42 [Andrew Or] Fix tests
f5be0f3 [Andrew Or] Use MEMORY_AND_DISK as the default local checkpoint level
a92657d [Andrew Or] Update a few comments
e58e3e3 [Andrew Or] Merge branch 'master' of github.com:apache/spark into local-checkpoint
4eb6eb1 [Andrew Or] Merge branch 'master' of github.com:apache/spark into local-checkpoint
1bbe154 [Andrew Or] Simplify LocalCheckpointRDD
48a9996 [Andrew Or] Avoid traversing dependency tree + rewrite tests
62aba3f [Andrew Or] Merge branch 'master' of github.com:apache/spark into local-checkpoint
db70dc2 [Andrew Or] Express local checkpointing through caching the original RDD
87d43c6 [Andrew Or] Merge branch 'master' of github.com:apache/spark into local-checkpoint
c449b38 [Andrew Or] Fix style
4a182f3 [Andrew Or] Add fine-grained tests for local checkpointing
53b363b [Andrew Or] Rename a few more awkwardly named methods (minor)
e4cf071 [Andrew Or] Simplify LocalCheckpointRDD + docs + clean ups
4880deb [Andrew Or] Fix style
d096c67 [Andrew Or] Fix mima
172cb66 [Andrew Or] Fix mima?
e53d964 [Andrew Or] Fix style
56831c5 [Andrew Or] Add a few warnings and clear exception messages
2e59646 [Andrew Or] Add local checkpoint clean up tests
4dbbab1 [Andrew Or] Refactor CheckpointSuite to test local checkpointing
4514dc9 [Andrew Or] Clean local checkpoint files through RDD cleanups
0477eec [Andrew Or] Rename a few methods with awkward names (minor)
2e902e5 [Andrew Or] First implementation of local checkpointing
8447454 [Andrew Or] Fix tests
4ac1896 [Andrew Or] Refactor checkpoint interface for modularity
This patch builds directly on #7820, which is largely written by tnachen. The only addition is one commit for cleaning up the code. There should be no functional differences between this and #7820.
Author: Timothy Chen <tnachen@gmail.com>
Author: Andrew Or <andrew@databricks.com>
Closes#7881 from andrewor14/tim-cleanup-mesos-shuffle and squashes the following commits:
8894f7d [Andrew Or] Clean up code
2a5fa10 [Andrew Or] Merge branch 'mesos_shuffle_clean' of github.com:tnachen/spark into tim-cleanup-mesos-shuffle
fadff89 [Timothy Chen] Address comments.
e4d0f1d [Timothy Chen] Clean up external shuffle data on driver exit with Mesos.
This pull request adds a destructAndCreateExternalSorter method to UnsafeFixedWidthAggregationMap. The new method does the following:
1. Creates a new external sorter UnsafeKVExternalSorter
2. Adds all the data into an in-memory sorter, sorts them
3. Spills the sorted in-memory data to disk
This method can be used to fallback to sort-based aggregation when under memory pressure.
The pull request also includes accounting fixes from JoshRosen.
TODOs (that can be done in follow-up PRs)
- [x] Address Josh's feedbacks from #7849
- [x] More documentation and test cases
- [x] Make sure we are doing memory accounting correctly with test cases (e.g. did we release the memory in BytesToBytesMap twice?)
- [ ] Look harder at possible memory leaks and exception handling
- [ ] Randomized tester for the KV sorter as well as the aggregation map
Author: Reynold Xin <rxin@databricks.com>
Author: Josh Rosen <joshrosen@databricks.com>
Closes#7860 from rxin/kvsorter and squashes the following commits:
986a58c [Reynold Xin] Bug fix.
599317c [Reynold Xin] Style fix and slightly more compact code.
fe7bd4e [Reynold Xin] Bug fixes.
fd71bef [Reynold Xin] Merge remote-tracking branch 'josh/large-records-in-sql-sorter' into kvsorter-with-josh-fix
3efae38 [Reynold Xin] More fixes and documentation.
45f1b09 [Josh Rosen] Ensure that spill files are cleaned up
f6a9bd3 [Reynold Xin] Josh feedback.
9be8139 [Reynold Xin] Remove testSpillFrequency.
7cbe759 [Reynold Xin] [SPARK-9531][SQL] UnsafeFixedWidthAggregationMap.destructAndCreateExternalSorter.
ae4a8af [Josh Rosen] Detect leaked unsafe memory in UnsafeExternalSorterSuite.
52f9b06 [Josh Rosen] Detect ShuffleMemoryManager leaks in UnsafeExternalSorter.
Dynamic allocation is a feature that allows a Spark application to scale the number of executors up and down dynamically based on the workload. Support was first introduced in YARN since 1.2, and then extended to Mesos coarse-grained mode recently. Today, it is finally supported in standalone mode as well!
I tested this locally and it works as expected. This is WIP because unit tests are coming.
Author: Andrew Or <andrew@databricks.com>
Closes#7532 from andrewor14/standalone-da and squashes the following commits:
b3c1736 [Andrew Or] Merge branch 'master' of github.com:apache/spark into standalone-da
879e928 [Andrew Or] Add end-to-end tests for standalone dynamic allocation
accc8f6 [Andrew Or] Address comments
ee686a8 [Andrew Or] Merge branch 'master' of github.com:apache/spark into standalone-da
c0a2c02 [Andrew Or] Fix build after merge conflict
24149eb [Andrew Or] Merge branch 'master' of github.com:apache/spark into standalone-da
2e762d6 [Andrew Or] Merge branch 'master' of github.com:apache/spark into standalone-da
6832bd7 [Andrew Or] Add tests for scheduling with executor limit
a82e907 [Andrew Or] Fix comments
0a8be79 [Andrew Or] Simplify logic by removing the worker blacklist
b7742af [Andrew Or] Merge branch 'master' of github.com:apache/spark into standalone-da
2eb5f3f [Andrew Or] Merge branch 'master' of github.com:apache/spark into standalone-da
1334e9a [Andrew Or] Fix MiMa
32abe44 [Andrew Or] Fix style
58cb06f [Andrew Or] Privatize worker blacklist for cleanliness
42ac215 [Andrew Or] Clean up comments and rewrite code for readability
49702d1 [Andrew Or] Clean up shuffle files after application exits
80047aa [Andrew Or] First working implementation
BytesToBytesMap current encodes key/value data in the following format:
```
8B key length, key data, 8B value length, value data
```
UnsafeExternalSorter, on the other hand, encodes data this way:
```
4B record length, data
```
As a result, we cannot pass records encoded by BytesToBytesMap directly into UnsafeExternalSorter for sorting. However, if we rearrange data slightly, we can then pass the key/value records directly into UnsafeExternalSorter:
```
4B key+value length, 4B key length, key data, value data
```
Author: Reynold Xin <rxin@databricks.com>
Closes#7845 from rxin/kvsort-rebase and squashes the following commits:
5716b59 [Reynold Xin] Fixed test.
2e62ccb [Reynold Xin] Updated BytesToBytesMap's data encoding to put the key first.
a51b641 [Reynold Xin] Added a KV sorter interface.
This patch adds support for entries larger than the default page size in BytesToBytesMap. These large rows are handled by allocating special overflow pages to hold individual entries.
In addition, this patch integrates BytesToBytesMap with the ShuffleMemoryManager:
- Move BytesToBytesMap from `unsafe` to `core` so that it can import `ShuffleMemoryManager`.
- Before allocating new data pages, ask the ShuffleMemoryManager to reserve the memory:
- `putNewKey()` now returns a boolean to indicate whether the insert succeeded or failed due to a lack of memory. The caller can use this value to respond to the memory pressure (e.g. by spilling).
- `UnsafeFixedWidthAggregationMap. getAggregationBuffer()` now returns `null` to signal failure due to a lack of memory.
- Updated all uses of these classes to handle these error conditions.
- Added new tests for allocating large records and for allocations which fail due to memory pressure.
- Extended the `afterAll()` test teardown methods to detect ShuffleMemoryManager leaks.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#7762 from JoshRosen/large-rows and squashes the following commits:
ae7bc56 [Josh Rosen] Fix compilation
82fc657 [Josh Rosen] Merge remote-tracking branch 'origin/master' into large-rows
34ab943 [Josh Rosen] Remove semi
31a525a [Josh Rosen] Integrate BytesToBytesMap with ShuffleMemoryManager.
626b33c [Josh Rosen] Move code to sql/core and spark/core packages so that ShuffleMemoryManager can be integrated
ec4484c [Josh Rosen] Move BytesToBytesMap from unsafe package to core.
642ed69 [Josh Rosen] Rename size to numElements
bea1152 [Josh Rosen] Add basic test.
2cd3570 [Josh Rosen] Remove accidental duplicated code
07ff9ef [Josh Rosen] Basic support for large rows in BytesToBytesMap.
Author: Reynold Xin <rxin@databricks.com>
Closes#7803 from rxin/SPARK-9458 and squashes the following commits:
5b032dc [Reynold Xin] Fix string.
b670dbb [Reynold Xin] [SPARK-9458][SPARK-9469][SQL] Code generate prefix computation in sorting & moves unsafe conversion out of TungstenSort.
As of today, StringPrefixComparator converts the long values back to byte arrays in order to compare them. This patch optimizes this to compare the longs directly, rather than turning the longs into byte arrays and comparing them byte by byte (unsigned).
This only works on little-endian architecture right now.
Author: Reynold Xin <rxin@databricks.com>
Closes#7765 from rxin/SPARK-9460 and squashes the following commits:
e4908cc [Reynold Xin] Stricter randomized tests.
4c8d094 [Reynold Xin] [SPARK-9460] Avoid byte array allocation in StringPrefixComparator.
In our existing sort prefix generation code, we use expression's eval method to generate the prefix, which results in object allocation for every prefix. We can use the specialized getters available on InternalRow directly to avoid the object allocation.
I also removed the FLOAT prefix, opting for converting float directly to double.
Author: Reynold Xin <rxin@databricks.com>
Closes#7763 from rxin/sort-prefix and squashes the following commits:
5dc2f06 [Reynold Xin] [SPARK-9458] Avoid object allocation in prefix generation.
We need to make page sizes configurable so we can reduce them in unit tests and increase them in real production workloads. These sizes are now controlled by a new configuration, `spark.buffer.pageSize`. The new default is 64 megabytes.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#7741 from JoshRosen/SPARK-9411 and squashes the following commits:
a43c4db [Josh Rosen] Fix pow
2c0eefc [Josh Rosen] Fix MAXIMUM_PAGE_SIZE_BYTES comment + value
bccfb51 [Josh Rosen] Lower page size to 4MB in TestHive
ba54d4b [Josh Rosen] Make UnsafeExternalSorter's page size configurable
0045aa2 [Josh Rosen] Make UnsafeShuffle's page size configurable
bc734f0 [Josh Rosen] Rename configuration
e614858 [Josh Rosen] Makes BytesToBytesMap page size configurable
Added a custom Kryo serializer for generic Avro records to reduce the network IO
involved during a shuffle. This compresses the schema and allows for users to
register their schemas ahead of time to further reduce traffic.
Currently Kryo tries to use its default serializer for generic Records, which will include
a lot of unneeded data in each record.
Author: Joseph Batchik <joseph.batchik@cloudera.com>
Author: Joseph Batchik <josephbatchik@gmail.com>
Closes#7004 from JDrit/Avro_serialization and squashes the following commits:
8158d51 [Joseph Batchik] updated per feedback
c0cf329 [Joseph Batchik] implemented @squito suggestion for SparkEnv
dd71efe [Joseph Batchik] fixed bug with serializing
1183a48 [Joseph Batchik] updated codec settings
fa9298b [Joseph Batchik] forgot a couple of fixes
c5fe794 [Joseph Batchik] implemented @squito suggestion
0f5471a [Joseph Batchik] implemented @squito suggestion to use a codec that is already in spark
6d1925c [Joseph Batchik] fixed to changes suggested by @squito
d421bf5 [Joseph Batchik] updated pom to removed versions
ab46d10 [Joseph Batchik] Changed Avro dependency to be similar to parent
f4ae251 [Joseph Batchik] fixed serialization error in that SparkConf cannot be serialized
2b545cc [Joseph Batchik] started working on fixes for pr
97fba62 [Joseph Batchik] Added a custom Kryo serializer for generic Avro records to reduce the network IO involved during a shuffle. This compresses the schema and allows for users to register their schemas ahead of time to further reduce traffic.
Spark's ShuffleMemoryManager and MemoryStore track memory on a per-thread basis, which causes problems in the handful of cases where we have tasks that use multiple threads. In PythonRDD, RRDD, ScriptTransformation, and PipedRDD we consume the input iterator in a separate thread in order to write it to an external process. As a result, these RDD's input iterators are consumed in a different thread than the thread that created them, which can cause problems in our memory allocation tracking. For example, if allocations are performed in one thread but deallocations are performed in a separate thread then memory may be leaked or we may get errors complaining that more memory was allocated than was freed.
I think that the right way to fix this is to change our accounting to be performed on a per-task instead of per-thread basis. Note that the current per-thread tracking has caused problems in the past; SPARK-3731 (#2668) fixes a memory leak in PythonRDD that was caused by this issue (that fix is no longer necessary as of this patch).
Author: Josh Rosen <joshrosen@databricks.com>
Closes#7734 from JoshRosen/memory-tracking-fixes and squashes the following commits:
b4b1702 [Josh Rosen] Propagate TaskContext to writer threads.
57c9b4e [Josh Rosen] Merge remote-tracking branch 'origin/master' into memory-tracking-fixes
ed25d3b [Josh Rosen] Address minor PR review comments
44f6497 [Josh Rosen] Fix long line.
7b0f04b [Josh Rosen] Fix ShuffleMemoryManagerSuite
f57f3f2 [Josh Rosen] More thread -> task changes
fa78ee8 [Josh Rosen] Move Executor's cleanup into Task so that TaskContext is defined when cleanup is performed
5e2f01e [Josh Rosen] Fix capitalization
1b0083b [Josh Rosen] Roll back fix in PySpark, which is no longer necessary
2e1e0f8 [Josh Rosen] Use TaskAttemptIds to track shuffle memory
c9e8e54 [Josh Rosen] Use TaskAttemptIds to track unroll memory
Currently there's no locality preference for container request in YARN mode, this will affect the performance if fetching data remotely, so here proposed to add locality in Yarn dynamic allocation mode.
Ping sryza, please help to review, thanks a lot.
Author: jerryshao <saisai.shao@intel.com>
Closes#6394 from jerryshao/SPARK-4352 and squashes the following commits:
d45fecb [jerryshao] Add documents
6c3fe5c [jerryshao] Fix bug
8db6c0e [jerryshao] Further address the comments
2e2b2cb [jerryshao] Fix rebase compiling problem
ce5f096 [jerryshao] Fix style issue
7f7df95 [jerryshao] Fix rebase issue
9ca9e07 [jerryshao] Code refactor according to comments
d3e4236 [jerryshao] Further address the comments
5e7a593 [jerryshao] Fix bug introduced code rebase
9ca7783 [jerryshao] Style changes
08317f9 [jerryshao] code and comment refines
65b2423 [jerryshao] Further address the comments
a27c587 [jerryshao] address the comment
27faabc [jerryshao] redundant code remove
9ce06a1 [jerryshao] refactor the code
f5ba27b [jerryshao] Style fix
2c6cc8a [jerryshao] Fix bug and add unit tests
0757335 [jerryshao] Consider the distribution of existed containers to recalculate the new container requests
0ad66ff [jerryshao] Fix compile bugs
1c20381 [jerryshao] Minor fix
5ef2dc8 [jerryshao] Add docs and improve the code
3359814 [jerryshao] Fix rebase and test bugs
0398539 [jerryshao] reinitialize the new implementation
67596d6 [jerryshao] Still fix the code
654e1d2 [jerryshao] Fix some bugs
45b1c89 [jerryshao] Further polish the algorithm
dea0152 [jerryshao] Enable node locality information in YarnAllocator
74bbcc6 [jerryshao] Support node locality for dynamic allocation initial commit
This patch fixes two bugs in UnsafeExternalSorter and UnsafeExternalRowSorter:
- UnsafeExternalSorter does not properly update freeSpaceInCurrentPage, which can cause it to write past the end of memory pages and trigger segfaults.
- UnsafeExternalRowSorter has a use-after-free bug when returning the last row from an iterator.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#7680 from JoshRosen/SPARK-9364 and squashes the following commits:
590f311 [Josh Rosen] null out row
f4cf91d [Josh Rosen] Fix use-after-free bug in UnsafeExternalRowSorter.
8abcf82 [Josh Rosen] Properly decrement freeSpaceInCurrentPage in UnsafeExternalSorter
This also fixes a small issue in the standalone Master that was uncovered by the new tests. For more detail, read the description of SPARK-9353.
Author: Andrew Or <andrew@databricks.com>
Closes#7668 from andrewor14/standalone-scheduling-tests and squashes the following commits:
d852faf [Andrew Or] Add tests + fix scheduling with memory limits
It's a thing used in test cases, but named Row. Pretty annoying because everytime I search for Row, it shows up before the Spark SQL Row, which is what a developer wants most of the time.
Author: Reynold Xin <rxin@databricks.com>
Closes#7638 from rxin/remove-row and squashes the following commits:
aeda52d [Reynold Xin] [SPARK-9305] Rename org.apache.spark.Row to Item.
See also comments on https://issues.apache.org/jira/browse/SPARK-9236
Author: François Garillot <francois@garillot.net>
Closes#7616 from huitseeker/issue/SPARK-9236 and squashes the following commits:
217f902 [François Garillot] [SPARK-9236] Make defaultPartitioner not reuse a parent RDD's partitioner if it has 0 partitions
When a task fails with an exception and also fails to properly clean up its managed memory, the `spark.unsafe.exceptionOnMemoryLeak` memory leak detection mechanism's exceptions will mask the original exception that caused the task to fail. We should throw the memory leak exception only if no other exception occurred.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#7603 from JoshRosen/SPARK-9266 and squashes the following commits:
c268cb5 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-9266
c1f0167 [Josh Rosen] Fix the error masking problem
448eae8 [Josh Rosen] Add regression test
Spark has an option called spark.localExecution.enabled; according to the docs:
> Enables Spark to run certain jobs, such as first() or take() on the driver, without sending tasks to the cluster. This can make certain jobs execute very quickly, but may require shipping a whole partition of data to the driver.
This feature ends up adding quite a bit of complexity to DAGScheduler, especially in the runLocallyWithinThread method, but as far as I know nobody uses this feature (I searched the mailing list and haven't seen any recent mentions of the configuration nor stacktraces including the runLocally method). As a step towards scheduler complexity reduction, I propose that we remove this feature and all code related to it for Spark 1.5.
This pull request simply brings #7484 up to date.
Author: Josh Rosen <joshrosen@databricks.com>
Author: Reynold Xin <rxin@databricks.com>
Closes#7585 from rxin/remove-local-exec and squashes the following commits:
84bd10e [Reynold Xin] Python fix.
1d9739a [Reynold Xin] Merge pull request #7484 from JoshRosen/remove-localexecution
eec39fa [Josh Rosen] Remove allowLocal(); deprecate user-facing uses of it.
b0835dc [Josh Rosen] Remove local execution code in DAGScheduler
8975d96 [Josh Rosen] Remove local execution tests.
ffa8c9b [Josh Rosen] Remove documentation for configuration
There are a few memory limits that people hit often and that we could
make higher, especially now that memory sizes have grown.
- spark.akka.frameSize: This defaults at 10 but is often hit for map
output statuses in large shuffles. This memory is not fully allocated
up-front, so we can just make this larger and still not affect jobs
that never sent a status that large. We increase it to 128.
- spark.executor.memory: Defaults at 512m, which is really small. We
increase it to 1g.
Author: Matei Zaharia <matei@databricks.com>
Closes#7586 from mateiz/configs and squashes the following commits:
ce0038a [Matei Zaharia] [SPARK-9244] Increase some memory defaults
This PR adds pagination for the task table to solve the scalability issue of the stage page. Here is the initial screenshot:
<img width="1347" alt="pagination" src="https://cloud.githubusercontent.com/assets/1000778/8679669/9e63863c-2a8e-11e5-94e4-994febcd6717.png">
The task table only shows 100 tasks. There is a page navigation above the table. Users can click the page navigation or type the page number to jump to another page. The table can be sorted by clicking the headers. However, unlike previous implementation, the sorting work is done in the server now. So clicking a table column to sort needs to refresh the web page.
Author: zsxwing <zsxwing@gmail.com>
Closes#7399 from zsxwing/task-table-pagination and squashes the following commits:
144f513 [zsxwing] Display the page navigation when the page number is out of range
a3eee22 [zsxwing] Add extra space for the error message
54c5b84 [zsxwing] Reset page to 1 if the user changes the page size
c2f7f39 [zsxwing] Add a text field to let users fill the page size
bad52eb [zsxwing] Display user-friendly error messages
410586b [zsxwing] Scroll down to the tasks table if the url contains any sort column
a0746d1 [zsxwing] Use expand-dag-viz-arrow-job and expand-dag-viz-arrow-stage instead of expand-dag-viz-arrow-true and expand-dag-viz-arrow-false
b123f67 [zsxwing] Use localStorage to remember the user's actions and replay them when loading the page
894a342 [zsxwing] Show the link cursor when hovering for headers and page links and other minor fix
4d4fecf [zsxwing] Address Carson's comments
d9285f0 [zsxwing] Add comments and fix the style
74285fa [zsxwing] Merge branch 'master' into task-table-pagination
db6c859 [zsxwing] Task table pagination for the Stage page
Author: Jacek Lewandowski <lewandowski.jacek@gmail.com>
Closes#5805 from jacek-lewandowski/SPARK-7171 and squashes the following commits:
ed20bda [Jacek Lewandowski] SPARK-7171: Added a method to retrieve metrics sources in TaskContext
JIRA: https://issues.apache.org/jira/browse/SPARK-9128
Currently, in `ClosureCleaner`, the outerclasses and objects are retrieved using two different methods. However, the logic of the two methods is the same, and we can get both the outerclasses and objects with only one method calling.
Author: Liang-Chi Hsieh <viirya@appier.com>
Closes#7459 from viirya/remove_extra_closurecleaner and squashes the following commits:
7c9858d [Liang-Chi Hsieh] For comments.
a096941 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into remove_extra_closurecleaner
2ec5ce1 [Liang-Chi Hsieh] Remove unnecessary methods.
4df5a51 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into remove_extra_closurecleaner
dc110d1 [Liang-Chi Hsieh] Add method to get outerclasses and objects at the same time.
This PR implements a JSON serializer and deserializer in the JSONProtocol to handle the (de)serialization of SparkListenerExecutorMetricsUpdate events. It also includes a unit test in the JSONProtocolSuite file. This was implemented to satisfy the improvement request in the JIRA issue SPARK-9036.
Author: Ben <benjaminpiering@gmail.com>
Closes#7555 from NamelessAnalyst/master and squashes the following commits:
fb4e3cc [Ben] Update JSON Protocol and tests
aa69517 [Ben] Update JSON Protocol and tests --Corrected Stage Attempt to Stage Attempt ID
33e5774 [Ben] Update JSON Protocol Tests
3f237e7 [Ben] Update JSON Protocol Tests
84ca798 [Ben] Update JSON Protocol Tests
cde57a0 [Ben] Update JSON Protocol Tests
8049600 [Ben] Update JSON Protocol Tests
c5bc061 [Ben] Update JSON Protocol Tests
6f25785 [Ben] Merge remote-tracking branch 'origin/master'
df2a609 [Ben] Update JSON Protocol
dcda80b [Ben] Update JSON Protocol
The shuffle code has gotten increasingly difficult to read as it has evolved, and many classes
have evolved significantly since they were originally created. The BlockStoreShuffleFetcher class
now serves little purpose other than to make the code more difficult to read; this commit moves its
functionality into the ShuffleBlockFetcherIterator class.
cc massie JoshRosen (Josh, this PR also removes the Try you pointed out as being confusing / not necessarily useful in a previous comment). Matt, would be helpful to know whether this will interfere in any negative ways with your new shuffle PR (I took a look and it seems like this should still cleanly integrate with your parquet work, but want to double check).
Author: Kay Ousterhout <kayousterhout@gmail.com>
Closes#7268 from kayousterhout/SPARK-8875 and squashes the following commits:
2b24a97 [Kay Ousterhout] Fixed DAGSchedulerSuite compile error
98a1831 [Kay Ousterhout] Merge remote-tracking branch 'upstream/master' into SPARK-8875
90f0e89 [Kay Ousterhout] Fixed broken test
14bfcbb [Kay Ousterhout] Last style fix
bc69d2b [Kay Ousterhout] Style improvements based on Josh's code review
ad3c8d1 [Kay Ousterhout] Better documentation for MapOutputTracker methods
0bc0e59 [Kay Ousterhout] [SPARK-8875] Remove BlockStoreShuffleFetcher class
This patch addresses an issue where queries that sorted float or double columns containing NaN values could fail with "Comparison method violates its general contract!" errors from TimSort. The root of this problem is that `NaN > anything`, `NaN == anything`, and `NaN < anything` all return `false`.
Per the design specified in SPARK-9079, we have decided that `NaN = NaN` should return true and that NaN should appear last when sorting in ascending order (i.e. it is larger than any other numeric value).
In addition to implementing these semantics, this patch also adds canonicalization of NaN values in UnsafeRow, which is necessary in order to be able to do binary equality comparisons on equal NaNs that might have different bit representations (see SPARK-9147).
Author: Josh Rosen <joshrosen@databricks.com>
Closes#7194 from JoshRosen/nan and squashes the following commits:
983d4fc [Josh Rosen] Merge remote-tracking branch 'origin/master' into nan
88bd73c [Josh Rosen] Fix Row.equals()
a702e2e [Josh Rosen] normalization -> canonicalization
a7267cf [Josh Rosen] Normalize NaNs in UnsafeRow
fe629ae [Josh Rosen] Merge remote-tracking branch 'origin/master' into nan
fbb2a29 [Josh Rosen] Fix NaN comparisons in BinaryComparison expressions
c1fd4fe [Josh Rosen] Fold NaN test into existing test framework
b31eb19 [Josh Rosen] Uncomment failing tests
7fe67af [Josh Rosen] Support NaN == NaN (SPARK-9145)
58bad2c [Josh Rosen] Revert "Compare rows' string representations to work around NaN incomparability."
fc6b4d2 [Josh Rosen] Update CodeGenerator
3998ef2 [Josh Rosen] Remove unused code
a2ba2e7 [Josh Rosen] Fix prefix comparision for NaNs
a30d371 [Josh Rosen] Compare rows' string representations to work around NaN incomparability.
6f03f85 [Josh Rosen] Fix bug in Double / Float ordering
42a1ad5 [Josh Rosen] Stop filtering NaNs in UnsafeExternalSortSuite
bfca524 [Josh Rosen] Change ordering so that NaN is maximum value.
8d7be61 [Josh Rosen] Update randomized test to use ScalaTest's assume()
b20837b [Josh Rosen] Add failing test for new NaN comparision ordering
5b88b2b [Josh Rosen] Fix compilation of CodeGenerationSuite
d907b5b [Josh Rosen] Merge remote-tracking branch 'origin/master' into nan
630ebc5 [Josh Rosen] Specify an ordering for NaN values.
9bf195a [Josh Rosen] Re-enable NaNs in CodeGenerationSuite to produce more regression tests
13fc06a [Josh Rosen] Add regression test for NaN sorting issue
f9efbb5 [Josh Rosen] Fix ORDER BY NULL
e7dc4fb [Josh Rosen] Add very generic test for ordering
7d5c13e [Josh Rosen] Add regression test for SPARK-8782 (ORDER BY NULL)
b55875a [Josh Rosen] Generate doubles and floats over entire possible range.
5acdd5c [Josh Rosen] Infinity and NaN are interesting.
ab76cbd [Josh Rosen] Move code to Catalyst package.
d2b4a4a [Josh Rosen] Add random data generator test utilities to Spark SQL.
https://issues.apache.org/jira/browse/SPARK-8103
cc kayousterhout (thanks for the extra test case)
Author: Imran Rashid <irashid@cloudera.com>
Author: Kay Ousterhout <kayousterhout@gmail.com>
Author: Imran Rashid <squito@users.noreply.github.com>
Closes#6750 from squito/SPARK-8103 and squashes the following commits:
fb3acfc [Imran Rashid] fix log msg
e01b7aa [Imran Rashid] fix some comments, style
584acd4 [Imran Rashid] simplify going from taskId to taskSetMgr
e43ac25 [Imran Rashid] Merge branch 'master' into SPARK-8103
6bc23af [Imran Rashid] update log msg
4470fa1 [Imran Rashid] rename
c04707e [Imran Rashid] style
88b61cc [Imran Rashid] add tests to make sure that TaskSchedulerImpl schedules correctly with zombie attempts
d7f1ef2 [Imran Rashid] get rid of activeTaskSets
a21c8b5 [Imran Rashid] Merge branch 'master' into SPARK-8103
906d626 [Imran Rashid] fix merge
109900e [Imran Rashid] Merge branch 'master' into SPARK-8103
c0d4d90 [Imran Rashid] Revert "Index active task sets by stage Id rather than by task set id"
f025154 [Imran Rashid] Merge pull request #2 from kayousterhout/imran_SPARK-8103
baf46e1 [Kay Ousterhout] Index active task sets by stage Id rather than by task set id
19685bb [Imran Rashid] switch to using latestInfo.attemptId, and add comments
a5f7c8c [Imran Rashid] remove comment for reviewers
227b40d [Imran Rashid] style
517b6e5 [Imran Rashid] get rid of SparkIllegalStateException
b2faef5 [Imran Rashid] faster check for conflicting task sets
6542b42 [Imran Rashid] remove extra stageAttemptId
ada7726 [Imran Rashid] reviewer feedback
d8eb202 [Imran Rashid] Merge branch 'master' into SPARK-8103
46bc26a [Imran Rashid] more cleanup of debug garbage
cb245da [Imran Rashid] finally found the issue ... clean up debug stuff
8c29707 [Imran Rashid] Merge branch 'master' into SPARK-8103
89a59b6 [Imran Rashid] more printlns ...
9601b47 [Imran Rashid] more debug printlns
ecb4e7d [Imran Rashid] debugging printlns
b6bc248 [Imran Rashid] style
55f4a94 [Imran Rashid] get rid of more random test case since kays tests are clearer
7021d28 [Imran Rashid] update test since listenerBus.waitUntilEmpty now throws an exception instead of returning a boolean
883fe49 [Kay Ousterhout] Unit tests for concurrent stages issue
6e14683 [Imran Rashid] unit test just to make sure we fail fast on concurrent attempts
06a0af6 [Imran Rashid] ignore for jenkins
c443def [Imran Rashid] better fix and simpler test case
28d70aa [Imran Rashid] wip on getting a better test case ...
a9bf31f [Imran Rashid] wip
This makes sure attempts are listed in the order they were executed, and that the
app's state matches the state of the most current attempt.
Author: Joshi <rekhajoshm@gmail.com>
Author: Rekha Joshi <rekhajoshm@gmail.com>
Closes#7253 from rekhajoshm/SPARK-8593 and squashes the following commits:
874dd80 [Joshi] History Server: updated order for multiple attempts(logcleaner)
716e0b1 [Joshi] History Server: updated order for multiple attempts(descending start time works everytime)
548c753 [Joshi] History Server: updated order for multiple attempts(descending start time works everytime)
83306a8 [Joshi] History Server: updated order for multiple attempts(descending start time)
b0fc922 [Joshi] History Server: updated order for multiple attempts(updated comment)
cc0fda7 [Joshi] History Server: updated order for multiple attempts(updated test)
304cb0b [Joshi] History Server: updated order for multiple attempts(reverted HistoryPage)
85024e8 [Joshi] History Server: updated order for multiple attempts
a41ac4b [Joshi] History Server: updated order for multiple attempts
ab65fa1 [Joshi] History Server: some attempt completed to work with showIncomplete
0be142d [Rekha Joshi] Merge pull request #3 from apache/master
106fd8e [Rekha Joshi] Merge pull request #2 from apache/master
e3677c9 [Rekha Joshi] Merge pull request #1 from apache/master
This PR includes the following changes:
1. Remove the thread local `Accumulators.localAccums`. Instead, all Accumulators in the executors will register with its TaskContext.
2. Add an internal flag to Accumulable. For internal Accumulators, their updates will be sent to the driver via heartbeats.
Author: zsxwing <zsxwing@gmail.com>
Closes#7448 from zsxwing/accumulators and squashes the following commits:
c24bc5b [zsxwing] Add comments
bd7dcf1 [zsxwing] Add an internal flag to Accumulable and send internal accumulator updates to the driver via heartbeats
**Symptom.** If an executor in an application times out, `HeartbeatReceiver` attempts to kill it. After this happens, however, the application never gets an executor back even when there are cluster resources available.
**Cause.** The issue is that `sc.killExecutor` automatically assumes that the application wishes to adjust its resource requirements permanently downwards. This is not the intention in `HeartbeatReceiver`, however, which simply wants a replacement for the expired executor.
**Fix.** Differentiate between the intention to kill and the intention to replace an executor with a fresh one. More details can be found in the commit message.
Author: Andrew Or <andrew@databricks.com>
Closes#7107 from andrewor14/heartbeat-no-kill and squashes the following commits:
1cd2cd7 [Andrew Or] Add regression test for SPARK-8119
25a347d [Andrew Or] Reuse more code in scheduler backend
31ebd40 [Andrew Or] Differentiate between kill and replace
Mesos supports framework authentication and role to be set per framework, which the role is used to identify the framework's role which impacts the sharing weight of resource allocation and optional authentication information to allow the framework to be connected to the master.
Author: Timothy Chen <tnachen@gmail.com>
Closes#4960 from tnachen/mesos_fw_auth and squashes the following commits:
0f9f03e [Timothy Chen] Fix review comments.
8f9488a [Timothy Chen] Fix rebase
f7fc2a9 [Timothy Chen] Add mesos role, auth and secret.
Example exception (new part at bottom, clearly demarcated):
```
org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in stage 0.0 (TID 0, localhost): java.lang.RuntimeException: uh-oh!
at org.apache.spark.scheduler.DAGSchedulerSuite$$anonfun$37$$anonfun$38$$anonfun$apply$mcJ$sp$2.apply(DAGSchedulerSuite.scala:880)
at org.apache.spark.scheduler.DAGSchedulerSuite$$anonfun$37$$anonfun$38$$anonfun$apply$mcJ$sp$2.apply(DAGSchedulerSuite.scala:880)
at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
at org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1640)
at org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1099)
at org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1099)
at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1777)
at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1777)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:63)
at org.apache.spark.scheduler.Task.run(Task.scala:70)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:213)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:744)
Driver stacktrace:
at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1298)
at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1289)
at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1288)
at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1288)
at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:755)
at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:755)
at scala.Option.foreach(Option.scala:236)
at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:755)
at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1509)
at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1470)
at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1459)
at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)
at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:560)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:1744)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:1762)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:1777)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:1791)
at org.apache.spark.rdd.RDD.count(RDD.scala:1099)
at org.apache.spark.scheduler.DAGSchedulerSuite$$anonfun$37$$anonfun$38.apply$mcJ$sp(DAGSchedulerSuite.scala:880)
at org.apache.spark.scheduler.DAGSchedulerSuite$$anonfun$37$$anonfun$38.apply(DAGSchedulerSuite.scala:880)
at org.apache.spark.scheduler.DAGSchedulerSuite$$anonfun$37$$anonfun$38.apply(DAGSchedulerSuite.scala:880)
at org.scalatest.Assertions$class.intercept(Assertions.scala:997)
at org.scalatest.FunSuite.intercept(FunSuite.scala:1555)
at org.apache.spark.scheduler.DAGSchedulerSuite$$anonfun$37.apply$mcV$sp(DAGSchedulerSuite.scala:879)
at org.apache.spark.scheduler.DAGSchedulerSuite$$anonfun$37.apply(DAGSchedulerSuite.scala:878)
at org.apache.spark.scheduler.DAGSchedulerSuite$$anonfun$37.apply(DAGSchedulerSuite.scala:878)
at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
at org.scalatest.Transformer.apply(Transformer.scala:22)
at org.scalatest.Transformer.apply(Transformer.scala:20)
at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166)
at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:42)
at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163)
at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175)
at org.apache.spark.scheduler.DAGSchedulerSuite.org$scalatest$BeforeAndAfter$$super$runTest(DAGSchedulerSuite.scala:70)
at org.scalatest.BeforeAndAfter$class.runTest(BeforeAndAfter.scala:200)
at org.apache.spark.scheduler.DAGSchedulerSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(DAGSchedulerSuite.scala:70)
at org.scalatest.BeforeAndAfterEach$class.runTest(BeforeAndAfterEach.scala:255)
at org.apache.spark.scheduler.DAGSchedulerSuite.runTest(DAGSchedulerSuite.scala:70)
at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208)
at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208)
at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:413)
at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:401)
at scala.collection.immutable.List.foreach(List.scala:318)
at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401)
at org.scalatest.SuperEngine.org$scalatest$SuperEngine$$runTestsInBranch(Engine.scala:396)
at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:483)
at org.scalatest.FunSuiteLike$class.runTests(FunSuiteLike.scala:208)
at org.scalatest.FunSuite.runTests(FunSuite.scala:1555)
at org.scalatest.Suite$class.run(Suite.scala:1424)
at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1555)
at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212)
at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212)
at org.scalatest.SuperEngine.runImpl(Engine.scala:545)
at org.scalatest.FunSuiteLike$class.run(FunSuiteLike.scala:212)
at org.apache.spark.scheduler.DAGSchedulerSuite.org$scalatest$BeforeAndAfter$$super$run(DAGSchedulerSuite.scala:70)
at org.scalatest.BeforeAndAfter$class.run(BeforeAndAfter.scala:241)
at org.apache.spark.scheduler.DAGSchedulerSuite.org$scalatest$BeforeAndAfterAll$$super$run(DAGSchedulerSuite.scala:70)
at org.scalatest.BeforeAndAfterAll$class.liftedTree1$1(BeforeAndAfterAll.scala:257)
at org.scalatest.BeforeAndAfterAll$class.run(BeforeAndAfterAll.scala:256)
at org.apache.spark.scheduler.DAGSchedulerSuite.run(DAGSchedulerSuite.scala:70)
at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:462)
at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:671)
at sbt.ForkMain$Run$2.call(ForkMain.java:294)
at sbt.ForkMain$Run$2.call(ForkMain.java:284)
at java.util.concurrent.FutureTask.run(FutureTask.java:262)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:744)
```
Author: Aaron Davidson <aaron@databricks.com>
Closes#7028 from aarondav/stack-trace and squashes the following commits:
4714664 [Aaron Davidson] [SPARK-8644] Include call site in SparkException stack traces thrown by job failures
Replace Akka Serialization with Spark Serializer and add unit tests.
Author: zsxwing <zsxwing@gmail.com>
Closes#7159 from zsxwing/remove-akka-serialization and squashes the following commits:
fc0fca3 [zsxwing] Merge branch 'master' into remove-akka-serialization
cf81a58 [zsxwing] Fix the code style
73251c6 [zsxwing] Add test scope
9ef4af9 [zsxwing] Add AkkaRpcEndpointRef.hashCode
433115c [zsxwing] Remove final
be3edb0 [zsxwing] Support deserializing RpcEndpointRef
ecec410 [zsxwing] Replace Akka Serialization with Spark Serializer
This pull request adds a Scalastyle regex rule which fails the style check if `Class.forName` is used directly. `Class.forName` always loads classes from the default / system classloader, but in a majority of cases, we should be using Spark's own `Utils.classForName` instead, which tries to load classes from the current thread's context classloader and falls back to the classloader which loaded Spark when the context classloader is not defined.
<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/7350)
<!-- Reviewable:end -->
Author: Josh Rosen <joshrosen@databricks.com>
Closes#7350 from JoshRosen/ban-Class.forName and squashes the following commits:
e3e96f7 [Josh Rosen] Merge remote-tracking branch 'origin/master' into ban-Class.forName
c0b7885 [Josh Rosen] Hopefully fix the last two cases
d707ba7 [Josh Rosen] Fix uses of Class.forName that I missed in my first cleanup pass
046470d [Josh Rosen] Merge remote-tracking branch 'origin/master' into ban-Class.forName
62882ee [Josh Rosen] Fix uses of Class.forName or add exclusion.
d9abade [Josh Rosen] Add stylechecker rule to ban uses of Class.forName
Replace #6634
This PR adds `SparkListenerBlockUpdated` to SparkListener so that it can monitor all block update infos that are sent to `BlockManagerMasaterEndpoint`, and also add new tables in the Storage tab to display the stream block infos.
![screen shot 2015-07-01 at 5 19 46 pm](https://cloud.githubusercontent.com/assets/1000778/8451562/c291a6ec-2016-11e5-890d-0afc174e1f8c.png)
Author: zsxwing <zsxwing@gmail.com>
Closes#6672 from zsxwing/SPARK-4072-2 and squashes the following commits:
df2c1d8 [zsxwing] Use xml query to check the xml elements
54d54af [zsxwing] Add unit tests for StoragePage
e29fb53 [zsxwing] Update as per TD's comments
ccbee07 [zsxwing] Fix the code style
6dc42b4 [zsxwing] Fix the replication level of blocks
450fad1 [zsxwing] Merge branch 'master' into SPARK-4072-2
1e9ef52 [zsxwing] Don't categorize by Executor ID
ca0ab69 [zsxwing] Fix the code style
3de2762 [zsxwing] Make object BlockUpdatedInfo private
e95b594 [zsxwing] Add 'Aggregated Stream Block Metrics by Executor' table
ba5d0d1 [zsxwing] Refactor the unit test to improve the readability
4bbe341 [zsxwing] Revert JsonProtocol and don't log SparkListenerBlockUpdated
b464dd1 [zsxwing] Add onBlockUpdated to EventLoggingListener
5ba014c [zsxwing] Fix the code style
0b1e47b [zsxwing] Add a developer api BlockUpdatedInfo
04838a9 [zsxwing] Fix the code style
2baa161 [zsxwing] Add unit tests
80f6c6d [zsxwing] Address comments
797ee4b [zsxwing] Display Streaming blocks in Streaming UI
BlockObjectWriter has only one concrete non-test class, DiskBlockObjectWriter. In order to simplify the code in preparation for other refactorings, I think that we should remove this base class and have only DiskBlockObjectWriter.
While at one time we may have planned to have multiple BlockObjectWriter implementations, that doesn't seem to have happened, so the extra abstraction seems unnecessary.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#7391 from JoshRosen/shuffle-write-interface-refactoring and squashes the following commits:
c418e33 [Josh Rosen] Fix compilation
5047995 [Josh Rosen] Fix comments
d5dc548 [Josh Rosen] Update references in comments
89dc797 [Josh Rosen] Rename test suite.
5755918 [Josh Rosen] Remove unnecessary val in case class
1607c91 [Josh Rosen] Merge BlockObjectWriter and DiskBlockObjectWriter
This PR enables SparkR to dynamically ship the SparkR binary package to the AM node in YARN cluster mode, thus it is no longer required that the SparkR package be installed on each worker node.
This PR uses the JDK jar tool to package the SparkR package, because jar is thought to be available on both Linux/Windows platforms where JDK has been installed.
This PR does not address the R worker involved in RDD API. Will address it in a separate JIRA issue.
This PR does not address SBT build. SparkR installation and packaging by SBT will be addressed in a separate JIRA issue.
R/install-dev.bat is not tested. shivaram , Could you help to test it?
Author: Sun Rui <rui.sun@intel.com>
Closes#6743 from sun-rui/SPARK-6797 and squashes the following commits:
ca63c86 [Sun Rui] Adjust MimaExcludes after rebase.
7313374 [Sun Rui] Fix unit test errors.
72695fb [Sun Rui] Fix unit test failures.
193882f [Sun Rui] Fix Mima test error.
fe25a33 [Sun Rui] Fix Mima test error.
35ecfa3 [Sun Rui] Fix comments.
c38a005 [Sun Rui] Unzipped SparkR binary package is still required for standalone and Mesos modes.
b05340c [Sun Rui] Fix scala style.
2ca5048 [Sun Rui] Fix comments.
1acefd1 [Sun Rui] Fix scala style.
0aa1e97 [Sun Rui] Fix scala style.
41d4f17 [Sun Rui] Add support for locating SparkR package for R workers required by RDD APIs.
49ff948 [Sun Rui] Invoke jar.exe with full path in install-dev.bat.
7b916c5 [Sun Rui] Use 'rem' consistently.
3bed438 [Sun Rui] Add a comment.
681afb0 [Sun Rui] Fix a bug that RRunner does not handle client deployment modes.
cedfbe2 [Sun Rui] [SPARK-6797][SPARKR] Add support for YARN cluster mode.
This patch adds a cache-friendly external sorter which operates on serialized bytes and uses this sorter to implement a new sort operator for Spark SQL and DataFrames.
### Overview of the new sorter
The new sorter design is inspired by [Alphasort](http://research.microsoft.com/pubs/68249/alphasort.doc) and implements a key-prefix optimization in order to improve the cache friendliness of the sort. In naive sort implementations, the sorting algorithm operates on an array of record pointers. To compare two records for ordering, the sorter must dereference these pointers, which likely involves random memory access, then compare the objects themselves.
![image](https://cloud.githubusercontent.com/assets/50748/8611390/3b1402ae-2675-11e5-8308-1a10bf347e6e.png)
In a key-prefix sort, the sort operates on an array which stores the record pointer alongside a prefix of the record's key. When comparing two records for ordering, the sorter first compares the the stored key prefixes. If the ordering can be determined from the key prefixes (i.e. the prefixes are unequal), then the sort can avoid directly comparing the records, avoiding random memory accesses and full record comparisons. For example, if we're sorting a list of strings then we can store the first 8 bytes of the UTF-8 encoded string as the key-prefix and can perform unsigned byte-at-a-time comparisons to determine the ordering of strings based on their prefixes, only resorting to full comparisons for strings that share a common prefix. In cases where the sort key can fit entirely in the space allotted for the key prefix (e.g. the sorting key is an integer), we completely avoid direct record comparison.
In this patch's implementation of key-prefix sorting, our sorter's internal array stores a 64-bit long and 64-bit pointer for each record being sorted. The key prefixes are generated by the user when inserting records into the sorter, which uses a user-defined comparison function for comparing them. The `PrefixComparators` object implements a set of comparators for many common types, including primitive numeric types and UTF-8 strings.
The actual sorting is implemented by `UnsafeInMemorySorter`. Most consumers will not use this directly, but instead will use `UnsafeExternalSorter`, a class which implements a sort that can spill to disk in response to memory pressure. Internally, `UnsafeExternalSorter` creates `UnsafeInMemorySorters` to perform sorting and uses `UnsafeSortSpillReader/Writer` to spill and read back runs of sorted records and `UnsafeSortSpillMerger` to merge multiple sorted spills into a single sorted iterator. This external sorter integrates with Spark's existing ShuffleMemoryManager for controlling spilling.
Many parts of this sorter's design are based on / copied from the more specialized external sort implementation that I designed for the new UnsafeShuffleManager write path; see #5868 for more details on that patch.
### Sorting rows in Spark SQL
For now, `UnsafeExternalSorter` is only used by Spark SQL, which uses it to implement a new sort operator, `UnsafeExternalSort`. This sort operator uses a SQL-specific class called `UnsafeExternalRowSorter` that configures an `UnsafeExternalSorter` to use prefix generators and comparators that operate on rows encoded in the UnsafeRow format that was designed for Project Tungsten.
I used some interesting unit-testing techniques to test this patch's SQL-specific components. `UnsafeExternalSortSuite` uses the SQL random data generators introduced in #7176 to test the UnsafeSort operator with all atomic types both with and without nullability and in both ascending and descending sort orders. `PrefixComparatorsSuite` contains a cool use of ScalaCheck + ScalaTest's `GeneratorDrivenPropertyChecks` in order to test UTF8String prefix comparison.
### Misc. additional improvements made in this patch
This patch made several miscellaneous improvements to related code in Spark SQL:
- The logic for selecting physical sort operator implementations, which was partially duplicated in both `Exchange` and `SparkStrategies, has now been consolidated into a `getSortOperator()` helper function in `SparkStrategies`.
- The `SparkPlanTest` unit testing helper trait has been extended with new methods for comparing the output produced by two different physical plans. This makes it easy to write tests which assert that two physical operator implementations should produce the same output. I also added a method for disabling the implicit sorting of outputs prior to comparing them, a change which is necessary in order to be able to write proper SparkPlan tests for sort operators.
### Tasks deferred to followup patches
While most of this patch's features are reasonably well-tested and complete, there are a number of tasks that are intentionally being deferred to followup patches:
- Add tests which mock the ShuffleMemoryManager to check that memory pressure properly triggers spilling (there are examples of this type of test in #5868).
- Add tests to ensure that spill files are properly cleaned up after errors. I'd like to do this in the context of a patch which introduces more general metrics for ensuring proper cleanup of tasks' temporary files; see https://issues.apache.org/jira/browse/SPARK-8966 for more details.
- Metrics integration: there are some open questions regarding how to track / report spill metrics for non-shuffle operations, so I've deferred most of the IO / shuffle metrics integration for now.
- Performance profiling.
<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/6444)
<!-- Reviewable:end -->
Author: Josh Rosen <joshrosen@databricks.com>
Closes#6444 from JoshRosen/sql-external-sort and squashes the following commits:
6beb467 [Josh Rosen] Remove a bunch of overloaded methods to avoid default args. issue
2bbac9c [Josh Rosen] Merge remote-tracking branch 'origin/master' into sql-external-sort
35dad9f [Josh Rosen] Make sortAnswers = false the default in SparkPlanTest
5135200 [Josh Rosen] Fix spill reading for large rows; add test
2f48777 [Josh Rosen] Add test and fix bug for sorting empty arrays
d1e28bc [Josh Rosen] Merge remote-tracking branch 'origin/master' into sql-external-sort
cd05866 [Josh Rosen] Fix scalastyle
3947fc1 [Josh Rosen] Merge remote-tracking branch 'origin/master' into sql-external-sort
d13ac55 [Josh Rosen] Hacky approach to copying of UnsafeRows for sort followed by limit.
845bea3 [Josh Rosen] Remove unnecessary zeroing of row conversion buffer
c56ec18 [Josh Rosen] Clean up final row copying code.
d31f180 [Josh Rosen] Re-enable NullType sorting test now that SPARK-8868 is fixed
844f4ca [Josh Rosen] Merge remote-tracking branch 'origin/master' into sql-external-sort
293f109 [Josh Rosen] Add missing license header.
f99a612 [Josh Rosen] Fix bugs in string prefix comparison.
9d00afc [Josh Rosen] Clean up prefix comparators for integral types
88aff18 [Josh Rosen] NULL_PREFIX has to be negative infinity for floating point types
613e16f [Josh Rosen] Test with larger data.
1d7ffaa [Josh Rosen] Somewhat hacky fix for descending sorts
08701e7 [Josh Rosen] Fix prefix comparison of null primitives.
b86e684 [Josh Rosen] Set global = true in UnsafeExternalSortSuite.
1c7bad8 [Josh Rosen] Make sorting of answers explicit in SparkPlanTest.checkAnswer().
b81a920 [Josh Rosen] Temporarily enable only the passing sort tests
5d6109d [Josh Rosen] Fix inconsistent handling / encoding of record lengths.
87b6ed9 [Josh Rosen] Fix critical issues in test which led to false negatives.
8d7fbe7 [Josh Rosen] Fixes to multiple spilling-related bugs.
82e21c1 [Josh Rosen] Force spilling in UnsafeExternalSortSuite.
88b72db [Josh Rosen] Test ascending and descending sort orders.
f27be09 [Josh Rosen] Fix tests by binding attributes.
0a79d39 [Josh Rosen] Revert "Undo part of a SparkPlanTest change in #7162 that broke my test."
7c3c864 [Josh Rosen] Undo part of a SparkPlanTest change in #7162 that broke my test.
9969c14 [Josh Rosen] Merge remote-tracking branch 'origin/master' into sql-external-sort
5822e6f [Josh Rosen] Fix test compilation issue
939f824 [Josh Rosen] Remove code gen experiment.
0dfe919 [Josh Rosen] Implement prefix sort for strings (albeit inefficiently).
66a813e [Josh Rosen] Prefix comparators for float and double
b310c88 [Josh Rosen] Integrate prefix comparators for Int and Long (others coming soon)
95058d9 [Josh Rosen] Add missing SortPrefixUtils file
4c37ba6 [Josh Rosen] Add tests for sorting on all primitive types.
6890863 [Josh Rosen] Fix memory leak on empty inputs.
d246e29 [Josh Rosen] Fix consideration of column types when choosing sort implementation.
6b156fb [Josh Rosen] Some WIP work on prefix comparison.
7f875f9 [Josh Rosen] Commit failing test demonstrating bug in handling objects in spills
41b8881 [Josh Rosen] Get UnsafeInMemorySorterSuite to pass (WIP)
90c2b6a [Josh Rosen] Update test name
6d6a1e6 [Josh Rosen] Centralize logic for picking sort operator implementations
9869ec2 [Josh Rosen] Clean up Exchange code a bit
82bb0ec [Josh Rosen] Fix IntelliJ complaint due to negated if condition
1db845a [Josh Rosen] Many more changes to harmonize with shuffle sorter
ebf9eea [Josh Rosen] Harmonization with shuffle's unsafe sorter
206bfa2 [Josh Rosen] Add some missing newlines at the ends of files
26c8931 [Josh Rosen] Back out some Hive changes that aren't needed anymore
62f0bb8 [Josh Rosen] Update to reflect SparkPlanTest changes
21d7d93 [Josh Rosen] Back out of BlockObjectWriter change
7eafecf [Josh Rosen] Port test to SparkPlanTest
d468a88 [Josh Rosen] Update for InternalRow refactoring
269cf86 [Josh Rosen] Back out SMJ operator change; isolate changes to selection of sort op.
1b841ca [Josh Rosen] WIP towards copying
b420a71 [Josh Rosen] Move most of the existing SMJ code into Java.
dfdb93f [Josh Rosen] SparkFunSuite change
73cc761 [Josh Rosen] Fix whitespace
9cc98f5 [Josh Rosen] Move more code to Java; fix bugs in UnsafeRowConverter length type.
c8792de [Josh Rosen] Remove some debug logging
dda6752 [Josh Rosen] Commit some missing code from an old git stash.
58f36d0 [Josh Rosen] Merge in a sketch of a unit test for the new sorter (now failing).
2bd8c9a [Josh Rosen] Import my original tests and get them to pass.
d5d3106 [Josh Rosen] WIP towards external sorter for Spark SQL.
Author: Jonathan Alter <jonalter@users.noreply.github.com>
Closes#7093 from jonalter/SPARK-7977 and squashes the following commits:
ccd44cc [Jonathan Alter] Changed println to log in ThreadingSuite
7fcac3e [Jonathan Alter] Reverting to println in ThreadingSuite
10724b6 [Jonathan Alter] Changing some printlns to logs in tests
eeec1e7 [Jonathan Alter] Merge branch 'master' of github.com:apache/spark into SPARK-7977
0b1dcb4 [Jonathan Alter] More println cleanup
aedaf80 [Jonathan Alter] Merge branch 'master' of github.com:apache/spark into SPARK-7977
925fd98 [Jonathan Alter] Merge branch 'master' of github.com:apache/spark into SPARK-7977
0c16fa3 [Jonathan Alter] Replacing some printlns with logs
45c7e05 [Jonathan Alter] Merge branch 'master' of github.com:apache/spark into SPARK-7977
5c8e283 [Jonathan Alter] Allowing println in audit-release examples
5b50da1 [Jonathan Alter] Allowing printlns in example files
ca4b477 [Jonathan Alter] Merge branch 'master' of github.com:apache/spark into SPARK-7977
83ab635 [Jonathan Alter] Fixing new printlns
54b131f [Jonathan Alter] Merge branch 'master' of github.com:apache/spark into SPARK-7977
1cd8a81 [Jonathan Alter] Removing some unnecessary comments and printlns
b837c3a [Jonathan Alter] Disallowing println
This is largely based on extracting the dynamic allocation parts from tnachen's #3861.
Author: Iulian Dragos <jaguarul@gmail.com>
Closes#4984 from dragos/issue/mesos-coarse-dynamicAllocation and squashes the following commits:
39df8cd [Iulian Dragos] Update tests to latest changes in core.
9d2c9fa [Iulian Dragos] Remove adjustment of executorLimitOption in doKillExecutors.
8b00f52 [Iulian Dragos] Latest round of reviews.
0cd00e0 [Iulian Dragos] Add persistent shuffle directory
15c45c1 [Iulian Dragos] Add dynamic allocation to the Spark coarse-grained scheduler.
Currently, the mesos scheduler only looks at the 'cpu' and 'mem' resources when trying to determine the usablility of a resource offer from a mesos slave node. It may be preferable for the user to be able to ensure that the spark jobs are only started on a certain set of nodes (based on attributes).
For example, If the user sets a property, let's say `spark.mesos.constraints` is set to `tachyon=true;us-east-1=false`, then the resource offers will be checked to see if they meet both these constraints and only then will be accepted to start new executors.
Author: Ankur Chauhan <achauhan@brightcove.com>
Closes#5563 from ankurcha/mesos_attribs and squashes the following commits:
902535b [Ankur Chauhan] Fix line length
d83801c [Ankur Chauhan] Update code as per code review comments
8b73f2d [Ankur Chauhan] Fix imports
c3523e7 [Ankur Chauhan] Added docs
1a24d0b [Ankur Chauhan] Expand scope of attributes matching to include all data types
482fd71 [Ankur Chauhan] Update access modifier to private[this] for offer constraints
5ccc32d [Ankur Chauhan] Fix nit pick whitespace
1bce782 [Ankur Chauhan] Fix nit pick whitespace
c0cbc75 [Ankur Chauhan] Use offer id value for debug message
7fee0ea [Ankur Chauhan] Add debug statements
fc7eb5b [Ankur Chauhan] Fix import codestyle
00be252 [Ankur Chauhan] Style changes as per code review comments
662535f [Ankur Chauhan] Incorporate code review comments + use SparkFunSuite
fdc0937 [Ankur Chauhan] Decline offers that did not meet criteria
67b58a0 [Ankur Chauhan] Add documentation for spark.mesos.constraints
63f53f4 [Ankur Chauhan] Update codestyle - uniform style for config values
02031e4 [Ankur Chauhan] Fix scalastyle warnings in tests
c09ed84 [Ankur Chauhan] Fixed the access modifier on offerConstraints val to private[mesos]
0c64df6 [Ankur Chauhan] Rename overhead fractions to memory_*, fix spacing
8cc1e8f [Ankur Chauhan] Make exception message more explicit about the source of the error
addedba [Ankur Chauhan] Added test case for malformed constraint string
ec9d9a6 [Ankur Chauhan] Add tests for parse constraint string
72fe88a [Ankur Chauhan] Fix up tests + remove redundant method override, combine utility class into new mesos scheduler util trait
92b47fd [Ankur Chauhan] Add attributes based constraints support to MesosScheduler
Latest changes after refactoring to the RPC layer. I rebased against trunk to make sure to get any recent changes since it had been a while. I wasn't crazy about the name `ConfigureTimeout` and `RpcTimeout` seemed to fit better, but I'm open to suggestions!
I ran most of the tests and they pass, but others would get stuck with "WARN TaskSchedulerImpl: Initial job has not accepted any resources". I think its just my machine, so I'd though I would push what I have anyway.
Still left to do:
* I only added a couple unit tests so far, there are probably some more cases to test
* Make sure all uses require a `RpcTimeout`
* Right now, both the `ask` and `Await.result` use the same timeout, should we differentiate between these in the TimeoutException message?
* I wrapped `Await.result` in `RpcTimeout`, should we also wrap `Await.ready`?
* Proper scoping of classes and methods
hardmettle, feel free to help out with any of these!
Author: Bryan Cutler <bjcutler@us.ibm.com>
Author: Harsh Gupta <harsh@Harshs-MacBook-Pro.local>
Author: BryanCutler <cutlerb@gmail.com>
Closes#6205 from BryanCutler/configTimeout-6980 and squashes the following commits:
46c8d48 [Bryan Cutler] [SPARK-6980] Changed RpcEnvSuite test to never reply instead of just sleeping, to avoid possible sync issues
06afa53 [Bryan Cutler] [SPARK-6980] RpcTimeout class extends Serializable, was causing error in MasterSuite
7bb70f1 [Bryan Cutler] Merge branch 'master' into configTimeout-6980
dbd5f73 [Bryan Cutler] [SPARK-6980] Changed RpcUtils askRpcTimeout and lookupRpcTimeout scope to private[spark] and improved deprecation warning msg
4e89c75 [Bryan Cutler] [SPARK-6980] Missed one usage of deprecated RpcUtils.askTimeout in YarnSchedulerBackend although it is not being used, and fixed SparkConfSuite UT to not use deprecated RpcUtils functions
6a1c50d [Bryan Cutler] [SPARK-6980] Minor cleanup of test case
7f4d78e [Bryan Cutler] [SPARK-6980] Fixed scala style checks
287059a [Bryan Cutler] [SPARK-6980] Removed extra import in AkkaRpcEnvSuite
3d8b1ff [Bryan Cutler] [SPARK-6980] Cleaned up imports in AkkaRpcEnvSuite
3a168c7 [Bryan Cutler] [SPARK-6980] Rewrote Akka RpcTimeout UTs in RpcEnvSuite
7636189 [Bryan Cutler] [SPARK-6980] Fixed call to askWithReply in DAGScheduler to use RpcTimeout - this was being compiled by auto-tupling and changing the message type of BlockManagerHeartbeat
be11c4e [Bryan Cutler] Merge branch 'master' into configTimeout-6980
039afed [Bryan Cutler] [SPARK-6980] Corrected import organization
218aa50 [Bryan Cutler] [SPARK-6980] Corrected issues from feedback
fadaf6f [Bryan Cutler] [SPARK-6980] Put back in deprecated RpcUtils askTimeout and lookupTimout to fix MiMa errors
fa6ed82 [Bryan Cutler] [SPARK-6980] Had to increase timeout on positive test case because a processor slowdown could trigger an Future TimeoutException
b05d449 [Bryan Cutler] [SPARK-6980] Changed constructor to use val duration instead of getter function, changed name of string property from conf to timeoutProp for consistency
c6cfd33 [Bryan Cutler] [SPARK-6980] Changed UT ask message timeout to explicitly intercept a SparkException
1394de6 [Bryan Cutler] [SPARK-6980] Moved MessagePrefix to createRpcTimeoutException directly
1517721 [Bryan Cutler] [SPARK-6980] RpcTimeout object scope should be private[spark]
2206b4d [Bryan Cutler] [SPARK-6980] Added unit test for ask then immediat awaitReply
1b9beab [Bryan Cutler] [SPARK-6980] Cleaned up import ordering
08f5afc [Bryan Cutler] [SPARK-6980] Added UT for constructing RpcTimeout with default value
d3754d1 [Bryan Cutler] [SPARK-6980] Added akkaConf to prevent dead letter logging
995d196 [Bryan Cutler] [SPARK-6980] Cleaned up import ordering, comments, spacing from PR feedback
7774d56 [Bryan Cutler] [SPARK-6980] Cleaned up UT imports
4351c48 [Bryan Cutler] [SPARK-6980] Added UT for addMessageIfTimeout, cleaned up UTs
1607a5f [Bryan Cutler] [SPARK-6980] Changed addMessageIfTimeout to PartialFunction, cleanup from PR comments
2f94095 [Bryan Cutler] [SPARK-6980] Added addMessageIfTimeout for when a Future is completed with TimeoutException
235919b [Bryan Cutler] [SPARK-6980] Resolved conflicts after master merge
c07d05c [Bryan Cutler] Merge branch 'master' into configTimeout-6980-tmp
b7fb99f [BryanCutler] Merge pull request #2 from hardmettle/configTimeoutUpdates_6980
4be3a8d [Harsh Gupta] Modifying loop condition to find property match
0ee5642 [Harsh Gupta] Changing the loop condition to halt at the first match in the property list for RpcEnv exception catch
f74064d [Harsh Gupta] Retrieving properties from property list using iterator and while loop instead of chained functions
a294569 [Bryan Cutler] [SPARK-6980] Added creation of RpcTimeout with Seq of property keys
23d2f26 [Bryan Cutler] [SPARK-6980] Fixed await result not being handled by RpcTimeout
49f9f04 [Bryan Cutler] [SPARK-6980] Minor cleanup and scala style fix
5b59a44 [Bryan Cutler] [SPARK-6980] Added some RpcTimeout unit tests
78a2c0a [Bryan Cutler] [SPARK-6980] Using RpcTimeout.awaitResult for future in AppClient now
97523e0 [Bryan Cutler] [SPARK-6980] Akka ask timeout description refactored to RPC layer
The existing test suite has a lot of duplicate code and doesn't even cover the most fundamental feature of the HeartbeatReceiver, which is expiring hosts that have not responded in a while.
This introduces manual clocks in `HeartbeatReceiver` and makes it respond to heartbeats only for registered executors. A few internal messages are moved to `receiveAndReply` to increase determinism of the tests so we don't have to rely on flaky constructs like `eventually`.
Author: Andrew Or <andrew@databricks.com>
Closes#7173 from andrewor14/heartbeat-receiver-tests and squashes the following commits:
4a903d6 [Andrew Or] Increase HeartReceiverSuite coverage and clean up
This patch rewrites the old checkpointing code in a way that is easier to understand. It also adds a guard against an invalid specification of checkpoint directory to provide a clearer error message. Most of the changes here are relatively minor.
Author: Andrew Or <andrew@databricks.com>
Closes#6968 from andrewor14/checkpoint-cleanup and squashes the following commits:
4ef8263 [Andrew Or] Use global synchronized instead
6f6fd84 [Andrew Or] Merge branch 'master' of github.com:apache/spark into checkpoint-cleanup
b1437ad [Andrew Or] Warn instead of throw
5484293 [Andrew Or] Merge branch 'master' of github.com:apache/spark into checkpoint-cleanup
7fb4af5 [Andrew Or] Guard against bad settings of checkpoint directory
691da98 [Andrew Or] Simplify checkpoint code / code style / comments
I've updated default values in comments, documentation, and in the command line builder to be 1g based on comments in the JIRA. I've also updated most usages to point at a single variable defined in the Utils.scala and JavaUtils.java files. This wasn't possible in all cases (R, shell scripts etc.) but usage in most code is now pointing at the same place.
Please let me know if I've missed anything.
Will the spark-shell use the value within the command line builder during instantiation?
Author: Ilya Ganelin <ilya.ganelin@capitalone.com>
Closes#7132 from ilganeli/SPARK-3071 and squashes the following commits:
4074164 [Ilya Ganelin] String fix
271610b [Ilya Ganelin] Merge branch 'SPARK-3071' of github.com:ilganeli/spark into SPARK-3071
273b6e9 [Ilya Ganelin] Test fix
fd67721 [Ilya Ganelin] Update JavaUtils.java
26cc177 [Ilya Ganelin] test fix
e5db35d [Ilya Ganelin] Fixed test failure
39732a1 [Ilya Ganelin] merge fix
a6f7deb [Ilya Ganelin] Created default value for DRIVER MEM in Utils that's now used in almost all locations instead of setting manually in each
09ad698 [Ilya Ganelin] Update SubmitRestProtocolSuite.scala
19b6f25 [Ilya Ganelin] Missed one doc update
2698a3d [Ilya Ganelin] Updated default value for driver memory
Otherwise other tests don't log anything useful...
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#7140 from vanzin/SPARK-3444 and squashes the following commits:
de14836 [Marcelo Vanzin] Better fix.
6cff13a [Marcelo Vanzin] [SPARK-3444] [core] Restore INFO level after log4j test.
This PR updates the rest Actors in core to RpcEndpoint.
Because there is no `ActorSelection` in RpcEnv, I changes the logic of `registerWithMaster` in Worker and AppClient to avoid blocking the message loop. These changes need to be reviewed carefully.
Author: zsxwing <zsxwing@gmail.com>
Closes#5392 from zsxwing/rpc-rewrite-part3 and squashes the following commits:
2de7bed [zsxwing] Merge branch 'master' into rpc-rewrite-part3
f12d943 [zsxwing] Address comments
9137b82 [zsxwing] Fix the code style
e734c71 [zsxwing] Merge branch 'master' into rpc-rewrite-part3
2d24fb5 [zsxwing] Fix the code style
5a82374 [zsxwing] Merge branch 'master' into rpc-rewrite-part3
fa47110 [zsxwing] Merge branch 'master' into rpc-rewrite-part3
72304f0 [zsxwing] Update the error strategy for AkkaRpcEnv
e56cb16 [zsxwing] Always send failure back to the sender
a7b86e6 [zsxwing] Use JFuture for java.util.concurrent.Future
aa34b9b [zsxwing] Fix the code style
bd541e7 [zsxwing] Merge branch 'master' into rpc-rewrite-part3
25a84d8 [zsxwing] Use ThreadUtils
060ff31 [zsxwing] Merge branch 'master' into rpc-rewrite-part3
dbfc916 [zsxwing] Improve the docs and comments
837927e [zsxwing] Merge branch 'master' into rpc-rewrite-part3
5c27f97 [zsxwing] Merge branch 'master' into rpc-rewrite-part3
fadbb9e [zsxwing] Fix the code style
6637e3c [zsxwing] Merge remote-tracking branch 'origin/master' into rpc-rewrite-part3
7fdee0e [zsxwing] Fix the return type to ExecutorService and ScheduledExecutorService
e8ad0a5 [zsxwing] Fix the code style
6b2a104 [zsxwing] Log error and use SparkExitCode.UNCAUGHT_EXCEPTION exit code
fbf3194 [zsxwing] Add Utils.newDaemonSingleThreadExecutor and newDaemonSingleThreadScheduledExecutor
b776817 [zsxwing] Update Master, Worker, Client, AppClient and related classes to use RpcEndpoint
Showing these applications may lead to weird behavior in the History Server. For old logs, if
the app ID is recorded later, you may end up with a duplicate entry. For new logs, the app might
be listed with a ".inprogress" suffix.
So ignore those, but still allow old applications that don't record app IDs at all (1.0 and 1.1) to be shown.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Author: Carson Wang <carson.wang@intel.com>
Closes#7097 from vanzin/SPARK-8372 and squashes the following commits:
a24eab2 [Marcelo Vanzin] Feedback.
112ae8f [Marcelo Vanzin] Merge branch 'master' into SPARK-8372
7b91b74 [Marcelo Vanzin] Handle logs generated by 1.0 and 1.1.
1eca3fe [Carson Wang] [SPARK-8372] History server shows incorrect information for application not started
Subset the enabled algorithms in an SSLOptions to the elements that are supported by the protocol provider.
Update the list of ciphers in the sample config to include modern algorithms, and specify both Oracle and IBM names. In practice the user would either specify their own chosen cipher suites, or specify none, and delegate the decision to the provider.
Author: Tim Ellison <t.p.ellison@gmail.com>
Closes#7043 from tellison/SSLEnhancements and squashes the following commits:
034efa5 [Tim Ellison] Ensure Java imports are grouped and ordered by package.
3797f8b [Tim Ellison] Remove unnecessary use of Option to improve clarity, and fix import style ordering.
4b5c89f [Tim Ellison] More robust SSL options processing.
This PR also includes re-ordering the order that repositories are used when resolving packages. User provided repositories will be prioritized.
cc andrewor14
Author: Burak Yavuz <brkyvz@gmail.com>
Closes#7089 from brkyvz/delete-prev-ivy-resolution and squashes the following commits:
a21f95a [Burak Yavuz] remove previous ivy resolution when using spark-submit
Hopefully, this suite will not be flaky anymore.
Author: Yin Huai <yhuai@databricks.com>
Closes#7027 from yhuai/SPARK-8567 and squashes the following commits:
c0167e2 [Yin Huai] Add sc.stop().
This is a follow up of #6404, the ScriptTransformation prints the error msg into stderr directly, probably be a disaster for application log.
Author: Cheng Hao <hao.cheng@intel.com>
Closes#6882 from chenghao-intel/verbose and squashes the following commits:
bfedd77 [Cheng Hao] revert the write
76ff46b [Cheng Hao] update the CircularBuffer
692b19e [Cheng Hao] check the process exitValue for ScriptTransform
47e0970 [Cheng Hao] Use the RedirectThread instead
1de771d [Cheng Hao] naming the threads in ScriptTransformation
8536e81 [Cheng Hao] disable the error message redirection for stderr
If `RDD.getPreferredLocations()` throws an exception it may crash the DAGScheduler and SparkContext. This patch addresses this by adding a try-catch block.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#7023 from JoshRosen/SPARK-8606 and squashes the following commits:
770b169 [Josh Rosen] Fix getPreferredLocations() DAGScheduler crash with try block.
44a9b55 [Josh Rosen] Add test of a buggy getPartitions() method
19aa9f7 [Josh Rosen] Add (failing) regression test for getPreferredLocations() DAGScheduler crash
This commit updates the shuffle read path to enable ShuffleReader implementations more control over the deserialization process.
The BlockStoreShuffleFetcher.fetch() method has been renamed to BlockStoreShuffleFetcher.fetchBlockStreams(). Previously, this method returned a record iterator; now, it returns an iterator of (BlockId, InputStream). Deserialization of records is now handled in the ShuffleReader.read() method.
This change creates a cleaner separation of concerns and allows implementations of ShuffleReader more flexibility in how records are retrieved.
Author: Matt Massie <massie@cs.berkeley.edu>
Author: Kay Ousterhout <kayousterhout@gmail.com>
Closes#6423 from massie/shuffle-api-cleanup and squashes the following commits:
8b0632c [Matt Massie] Minor Scala style fixes
d0a1b39 [Matt Massie] Merge pull request #1 from kayousterhout/massie_shuffle-api-cleanup
290f1eb [Kay Ousterhout] Added test for HashShuffleReader.read()
5186da0 [Kay Ousterhout] Revert "Add test to ensure HashShuffleReader is freeing resources"
f98a1b9 [Matt Massie] Add test to ensure HashShuffleReader is freeing resources
a011bfa [Matt Massie] Use PrivateMethodTester on check that delegate stream is closed
4ea1712 [Matt Massie] Small code cleanup for readability
7429a98 [Matt Massie] Update tests to check that BufferReleasingStream is closing delegate InputStream
f458489 [Matt Massie] Remove unnecessary map() on return Iterator
4abb855 [Matt Massie] Consolidate metric code. Make it clear why InterrubtibleIterator is needed.
5c30405 [Matt Massie] Return visibility of BlockStoreShuffleFetcher to private[hash]
7eedd1d [Matt Massie] Small Scala import cleanup
28f8085 [Matt Massie] Small import nit
f93841e [Matt Massie] Update shuffle read metrics in ShuffleReader instead of BlockStoreShuffleFetcher.
7e8e0fe [Matt Massie] Minor Scala style fixes
01e8721 [Matt Massie] Explicitly cast iterator in branches for type clarity
7c8f73e [Matt Massie] Close Block InputStream immediately after all records are read
208b7a5 [Matt Massie] Small code style changes
b70c945 [Matt Massie] Make BlockStoreShuffleFetcher visible to shuffle package
19135f2 [Matt Massie] [SPARK-7884] Allow Spark shuffle APIs to be more customizable
Author: Holden Karau <holden@pigscanfly.ca>
Closes#6918 from holdenk/SPARK-8498-fix-npe-in-errorhandling-path-in-unsafeshuffle-writer and squashes the following commits:
f807832 [Holden Karau] Log error if we can't throw it
855f9aa [Holden Karau] Spelling - not my strongest suite. Fix Propegates to Propagates.
039d620 [Holden Karau] Add missing closeandwriteoutput
30e558d [Holden Karau] go back to try/finally
e503b8c [Holden Karau] Improve the test to ensure we aren't masking the underlying exception
ae0b7a7 [Holden Karau] Fix the test
2e6abf7 [Holden Karau] Be more cautious when cleaning up during failed write and re-throw user exceptions
This patch also reenables the tests. Now that we have access to the log4j logs it should be easier to debug the flakiness.
yhuai brkyvz
Author: Andrew Or <andrew@databricks.com>
Closes#6886 from andrewor14/spark-submit-suite-fix and squashes the following commits:
3f99ff1 [Andrew Or] Move destroy to finally block
9a62188 [Andrew Or] Re-enable ignored tests
2382672 [Andrew Or] Check for exit code
This PR solves three SerializationDebugger issues.
* SPARK-7180 - SerializationDebugger fails with ArrayOutOfBoundsException
* SPARK-8090 - SerializationDebugger does not handle classes with writeReplace correctly
* SPARK-8091 - SerializationDebugger does not handle classes with writeObject method
The solutions for each are explained as follows
* SPARK-7180 - The wrong slot desc was used for getting the value of the fields in the object being tested.
* SPARK-8090 - Test the type of the replaced object.
* SPARK-8091 - Use a dummy ObjectOutputStream to collect all the objects written by the writeObject() method, and then test those objects as usual.
I also added more tests in the testsuite to increase code coverage. For example, added tests for cases where there are not serializability issues.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#6625 from tdas/SPARK-7180 and squashes the following commits:
c7cb046 [Tathagata Das] Addressed comments on docs
ae212c8 [Tathagata Das] Improved docs
304c97b [Tathagata Das] Fixed build error
26b5179 [Tathagata Das] more tests.....92% line coverage
7e2fdcf [Tathagata Das] Added more tests
d1967fb [Tathagata Das] Added comments.
da75d34 [Tathagata Das] Removed unnecessary lines.
50a608d [Tathagata Das] Fixed bugs and added support for writeObject
Dependencies of artifacts in the local ivy cache were not being resolved properly. The dependencies were not being picked up. Now they should be.
cc andrewor14
Author: Burak Yavuz <brkyvz@gmail.com>
Closes#6788 from brkyvz/local-ivy-fix and squashes the following commits:
2875bf4 [Burak Yavuz] fix temp dir bug
48cc648 [Burak Yavuz] improve deletion
a69e3e6 [Burak Yavuz] delete cache before test as well
0037197 [Burak Yavuz] fix merge conflicts
f60772c [Burak Yavuz] use different folder for m2 cache during testing
b6ef038 [Burak Yavuz] [SPARK-8095] Resolve dependencies of Spark Packages in local ivy cache
The history server may show an incorrect App ID for an incomplete application like <App ID>.inprogress. This app info will never disappear even after the app is completed.
![incorrectappinfo](https://cloud.githubusercontent.com/assets/9278199/8156147/2a10fdbe-137d-11e5-9620-c5b61d93e3c1.png)
The cause of the issue is that a log path name is used as the app id when app id cannot be got during replay.
Author: Carson Wang <carson.wang@intel.com>
Closes#6827 from carsonwang/SPARK-8372 and squashes the following commits:
cdbb089 [Carson Wang] Fix code style
3e46b35 [Carson Wang] Update code style
90f5dde [Carson Wang] Add a unit test
d8c9cd0 [Carson Wang] Replaying events only return information when app is started
The problem occurs because the position mask `0xEFFFFFF` is incorrect. It has zero 25th bit, so when capacity grows beyond 2^24, `OpenHashMap` calculates incorrect index of value in `_values` array.
I've also added a size check in `rehash()`, so that it fails instead of reporting invalid item indices.
Author: Vyacheslav Baranov <slavik.baranov@gmail.com>
Closes#6763 from SlavikBaranov/SPARK-8309 and squashes the following commits:
8557445 [Vyacheslav Baranov] Resolved review comments
4d5b954 [Vyacheslav Baranov] Resolved review comments
eaf1e68 [Vyacheslav Baranov] Fixed failing test
f9284fd [Vyacheslav Baranov] Resolved review comments
3920656 [Vyacheslav Baranov] SPARK-8309: Support for more than 12M items in OpenHashMap
Env variables are not visible to non-Spark users, based on suggestion from vanzin.
Author: Kan Zhang <kzhang@apache.org>
Closes#6774 from kanzhang/env and squashes the following commits:
5dd84c6 [Kan Zhang] remove auth secret conf from initial set up for executors
90cb7d2 [Kan Zhang] always filter out auth secret
af4d89d [Kan Zhang] minor refactering
e88993e [Kan Zhang] pass auth secret to executors via env variable
This change has two parts.
The first one gets rid of "ReflectionMagic". That worked well for the differences between 0.12 and
0.13, but breaks in 0.14, since some of the APIs that need to be used have primitive types. I could
not figure out a way to make that class work with primitive types. So instead I wrote some shims
(I can already hear the collective sigh) that find the appropriate methods via reflection. This should
be faster since the method instances are cached, and the code is not much uglier than before,
with the advantage that all the ugliness is local to one file (instead of multiple switch statements on
the version being used scattered in ClientWrapper).
The second part is simple: add code to handle Hive 0.14. A few new methods had to be added
to the new shims.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#6627 from vanzin/SPARK-8065 and squashes the following commits:
3fa4270 [Marcelo Vanzin] Indentation style.
4b8a3d4 [Marcelo Vanzin] Fix dep exclusion.
be3d0cc [Marcelo Vanzin] Merge branch 'master' into SPARK-8065
ca3fb1e [Marcelo Vanzin] Merge branch 'master' into SPARK-8065
b43f13e [Marcelo Vanzin] Since exclusions seem to work, clean up some of the code.
73bd161 [Marcelo Vanzin] Botched merge.
d2ddf01 [Marcelo Vanzin] Comment about excluded dep.
0c929d1 [Marcelo Vanzin] Merge branch 'master' into SPARK-8065
2c3c02e [Marcelo Vanzin] Try to fix tests by adding support for exclusions.
0a03470 [Marcelo Vanzin] Try to fix tests by upgrading calcite dependency.
13b2dfa [Marcelo Vanzin] Fix NPE.
6439d88 [Marcelo Vanzin] Minor style thing.
69b017b [Marcelo Vanzin] Style.
a21cad8 [Marcelo Vanzin] Part II: Add shims / version for Hive 0.14.
ae98c87 [Marcelo Vanzin] PART I: Get rid of reflection magic.
This patch updates two pieces of logic that are related to handling of keyOrderings in ShuffleDependencies:
- The Tungsten ShuffleManager falls back to regular SortShuffleManager whenever the shuffle dependency specifies a key ordering, but technically we only need to fall back when an aggregator is also specified. This patch updates the fallback logic to reflect this so that the Tungsten optimizations can apply to more workloads.
- The SQL Exchange operator performs defensive copying of shuffle inputs when a key ordering is specified, but this is unnecessary. The copying was added to guard against cases where ExternalSorter would buffer non-serialized records in memory. When ExternalSorter is configured without an aggregator, it uses the following logic to determine whether to buffer records in a serialized or deserialized format:
```scala
private val useSerializedPairBuffer =
ordering.isEmpty &&
conf.getBoolean("spark.shuffle.sort.serializeMapOutputs", true) &&
ser.supportsRelocationOfSerializedObjects
```
The `newOrdering.isDefined` branch in `ExternalSorter.needToCopyObjectsBeforeShuffle`, removed by this patch, is not necessary:
- It was checked even if we weren't using sort-based shuffle, but this was unnecessary because only SortShuffleManager performs map-side sorting.
- Map-side sorting during shuffle writing is only performed for shuffles that perform map-side aggregation as part of the shuffle (to see this, look at how SortShuffleWriter constructs ExternalSorter). Since SQL never pushes aggregation into Spark's shuffle, we can guarantee that both the aggregator and ordering will be empty and Spark SQL always uses serializers that support relocation, so sort-shuffle will use the serialized pair buffer unless the user has explicitly disabled it via the SparkConf feature-flag. Therefore, I think my optimization in Exchange should be safe.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#6773 from JoshRosen/SPARK-8319 and squashes the following commits:
7a14129 [Josh Rosen] Revise comments; add handler to guard against future ShuffleManager implementations
07bb2c9 [Josh Rosen] Update comment to clarify circumstances under which shuffle operates on serialized records
269089a [Josh Rosen] Avoid unnecessary copy in SQL Exchange
34e526e [Josh Rosen] Enable Tungsten shuffle for non-agg shuffles w/ key orderings