Commit graph

5286 commits

Author SHA1 Message Date
Davies Liu 37bc203c8d [SPARK-13210][SQL] catch OOM when allocate memory and expand array
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.
2016-02-08 12:09:20 -08:00
Tommy YU 81da3bee66 [SPARK-5865][API DOC] Add doc warnings for methods that return local data structures
rxin srowen
I work out note message for rdd.take function, please help to review.

If it's fine, I can apply to all other function later.

Author: Tommy YU <tummyyu@163.com>

Closes #10874 from Wenpei/spark-5865-add-warning-for-localdatastructure.
2016-02-06 17:29:09 +00:00
Davies Liu 4f28291f85 [HOTFIX] fix float part of avgRate 2016-02-05 22:40:40 -08:00
Jakob Odersky 6883a5120c [SPARK-13171][CORE] Replace future calls with Future
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.
2016-02-05 19:00:12 -08:00
Luc Bourlier 0bb5b73387 [SPARK-13002][MESOS] Send initial request of executors for dyn allocation
Fix for [SPARK-13002](https://issues.apache.org/jira/browse/SPARK-13002) about the initial number of executors when running with dynamic allocation on Mesos.
Instead of fixing it just for the Mesos case, made the change in `ExecutorAllocationManager`. It is already driving the number of executors running on Mesos, only no the initial value.

The `None` and `Some(0)` are internal details on the computation of resources to reserved, in the Mesos backend scheduler. `executorLimitOption` has to be initialized correctly, otherwise the Mesos backend scheduler will, either, create to many executors at launch, or not create any executors and not be able to recover from this state.

Removed the 'special case' description in the doc. It was not totally accurate, and is not needed anymore.

This doesn't fix the same problem visible with Spark standalone. There is no straightforward way to send the initial value in standalone mode.

Somebody knowing this part of the yarn support should review this change.

Author: Luc Bourlier <luc.bourlier@typesafe.com>

Closes #11047 from skyluc/issue/initial-dyn-alloc-2.
2016-02-05 14:37:42 -08:00
Jakob Odersky 352102ed0b [SPARK-13208][CORE] Replace use of Pairs with Tuple2s
Another trivial deprecation fix for Scala 2.11

Author: Jakob Odersky <jakob@odersky.com>

Closes #11089 from jodersky/SPARK-13208.
2016-02-04 22:22:41 -08:00
Raafat Akkad 6dbfc40776 [SPARK-13052] waitingApps metric doesn't show the number of apps currently in the WAITING state
Author: Raafat Akkad <raafat.akkad@gmail.com>

Closes #10959 from RaafatAkkad/master.
2016-02-04 16:09:31 -08:00
Andrew Or 7a4b37f02c [HOTFIX] Fix style violation caused by c756bda 2016-02-04 12:47:32 -08:00
Andrew Or c756bda477 [SPARK-12330][MESOS][HOTFIX] Rename timeout config
The config already describes time and accepts a general format
that is not restricted to ms. This commit renames the internal
config to use a format that's consistent in Spark.
2016-02-04 12:04:54 -08:00
Andrew Or 15205da817 [SPARK-13053][TEST] Unignore tests in InternalAccumulatorSuite
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.
2016-02-04 10:34:43 -08:00
Andrew Or 4120bcbaff [SPARK-13162] Standalone mode does not respect initial executors
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.
2016-02-04 10:32:32 -08:00
Holden Karau 62a7c28388 [SPARK-13164][CORE] Replace deprecated synchronized buffer in core
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.
2016-02-04 10:29:38 -08:00
Charles Allen 2eaeafe8a2 [SPARK-12330][MESOS] Fix mesos coarse mode cleanup
In the current implementation the mesos coarse scheduler does not wait for the mesos tasks to complete before ending the driver. This causes a race where the task has to finish cleaning up before the mesos driver terminates it with a SIGINT (and SIGKILL after 3 seconds if the SIGINT doesn't work).

This PR causes the mesos coarse scheduler to wait for the mesos tasks to finish (with a timeout defined by `spark.mesos.coarse.shutdown.ms`)

This PR also fixes a regression caused by [SPARK-10987] whereby submitting a shutdown causes a race between the local shutdown procedure and the notification of the scheduler driver disconnection. If the scheduler driver disconnection wins the race, the coarse executor incorrectly exits with status 1 (instead of the proper status 0)

With this patch the mesos coarse scheduler terminates properly, the executors clean up, and the tasks are reported as `FINISHED` in the Mesos console (as opposed to `KILLED` in < 1.6 or `FAILED` in 1.6 and later)

Author: Charles Allen <charles@allen-net.com>

Closes #10319 from drcrallen/SPARK-12330.
2016-02-04 10:27:25 -08:00
Liang-Chi Hsieh d39087147f [SPARK-13113] [CORE] Remove unnecessary bit operation when decoding page number
JIRA: https://issues.apache.org/jira/browse/SPARK-13113

As we shift bits right, looks like the bitwise AND operation is unnecessary.

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

Closes #11002 from viirya/improve-decodepagenumber.
2016-02-03 23:17:51 -08:00
Holden Karau a8e2ba776b [SPARK-13152][CORE] Fix task metrics deprecation warning
Make an internal non-deprecated version of incBytesRead and incRecordsRead so we don't have unecessary deprecation warnings in our build.

Right now incBytesRead and incRecordsRead are marked as deprecated and for internal use only. We should make private[spark] versions which are not deprecated and switch to those internally so as to not clutter up the warning messages when building.

cc andrewor14 who did the initial deprecation

Author: Holden Karau <holden@us.ibm.com>

Closes #11056 from holdenk/SPARK-13152-fix-task-metrics-deprecation-warnings.
2016-02-03 17:43:14 -08:00
Davies Liu de0914522f [SPARK-13131] [SQL] Use best and average time in benchmark
Best time is stabler than average time, also added a column for nano seconds per row (which could be used to estimate contributions of each components in a query).

Having best time and average time together for more information (we can see kind of variance).

rate, time per row and relative are all calculated using best time.

The result looks like this:
```
Intel(R) Core(TM) i7-4558U CPU  2.80GHz
rang/filter/sum:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
-------------------------------------------------------------------------------------------
rang/filter/sum codegen=false          14332 / 16646         36.0          27.8       1.0X
rang/filter/sum codegen=true              845 /  940        620.0           1.6      17.0X
```

Author: Davies Liu <davies@databricks.com>

Closes #11018 from davies/gen_bench.
2016-02-03 17:07:27 -08:00
Alex Bozarth 3221eddb8f [SPARK-3611][WEB UI] Show number of cores for each executor in application web UI
Added a Cores column in the Executors UI

Author: Alex Bozarth <ajbozart@us.ibm.com>

Closes #11039 from ajbozarth/spark3611.
2016-02-03 15:53:10 -08:00
Shixiong Zhu 335f10edad [SPARK-7997][CORE] Add rpcEnv.awaitTermination() back to SparkEnv
`rpcEnv.awaitTermination()` was not added in #10854 because some Streaming Python tests hung forever.

This patch fixed the hung issue and added rpcEnv.awaitTermination() back to SparkEnv.

Previously, Streaming Kafka Python tests shutdowns the zookeeper server before stopping StreamingContext. Then when stopping StreamingContext, KafkaReceiver may be hung due to https://issues.apache.org/jira/browse/KAFKA-601, hence, some thread of RpcEnv's Dispatcher cannot exit and rpcEnv.awaitTermination is hung.The patch just changed the shutdown order to fix it.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11031 from zsxwing/awaitTermination.
2016-02-02 21:13:54 -08:00
Adam Budde ff71261b65 [SPARK-13122] Fix race condition in MemoryStore.unrollSafely()
https://issues.apache.org/jira/browse/SPARK-13122

A race condition can occur in MemoryStore's unrollSafely() method if two threads that
return the same value for currentTaskAttemptId() execute this method concurrently. This
change makes the operation of reading the initial amount of unroll memory used, performing
the unroll, and updating the associated memory maps atomic in order to avoid this race
condition.

Initial proposed fix wraps all of unrollSafely() in a memoryManager.synchronized { } block. A cleaner approach might be introduce a mechanism that synchronizes based on task attempt ID. An alternative option might be to track unroll/pending unroll memory based on block ID rather than task attempt ID.

Author: Adam Budde <budde@amazon.com>

Closes #11012 from budde/master.
2016-02-02 19:35:33 -08:00
felixcheung 0df3cfb8ab [SPARK-12790][CORE] Remove HistoryServer old multiple files format
Removed isLegacyLogDirectory code path and updated tests
andrewor14

Author: felixcheung <felixcheung_m@hotmail.com>

Closes #10860 from felixcheung/historyserverformat.
2016-02-01 16:55:21 -08:00
Sean Owen 715a19d56f [SPARK-12637][CORE] Print stage info of finished stages properly
Improve printing of StageInfo in onStageCompleted

See also https://github.com/apache/spark/pull/10585

Author: Sean Owen <sowen@cloudera.com>

Closes #10922 from srowen/SPARK-12637.
2016-02-01 16:23:17 -08:00
Iulian Dragos c9b89a0a09 [SPARK-12979][MESOS] Don’t resolve paths on the local file system in Mesos scheduler
The driver filesystem is likely different from where the executors will run, so resolving paths (and symlinks, etc.) will lead to invalid paths on executors.

Author: Iulian Dragos <jaguarul@gmail.com>

Closes #10923 from dragos/issue/canonical-paths.
2016-02-01 13:38:38 -08:00
Nilanjan Raychaudhuri a41b68b954 [SPARK-12265][MESOS] Spark calls System.exit inside driver instead of throwing exception
This takes over #10729 and makes sure that `spark-shell` fails with a proper error message. There is a slight behavioral change: before this change `spark-shell` would exit, while now the REPL is still there, but `sc` and `sqlContext` are not defined and the error is visible to the user.

Author: Nilanjan Raychaudhuri <nraychaudhuri@gmail.com>
Author: Iulian Dragos <jaguarul@gmail.com>

Closes #10921 from dragos/pr/10729.
2016-02-01 13:33:24 -08:00
Timothy Chen 51b03b71ff [SPARK-12463][SPARK-12464][SPARK-12465][SPARK-10647][MESOS] Fix zookeeper dir with mesos conf and add docs.
Fix zookeeper dir configuration used in cluster mode, and also add documentation around these settings.

Author: Timothy Chen <tnachen@gmail.com>

Closes #10057 from tnachen/fix_mesos_dir.
2016-02-01 12:45:02 -08:00
Shixiong Zhu 6075573a93 [SPARK-6847][CORE][STREAMING] Fix stack overflow issue when updateStateByKey is followed by a checkpointed dstream
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.
2016-02-01 11:02:17 -08:00
Josh Rosen 289373b28c [SPARK-6363][BUILD] Make Scala 2.11 the default Scala version
This patch changes Spark's build to make Scala 2.11 the default Scala version. To be clear, this does not mean that Spark will stop supporting Scala 2.10: users will still be able to compile Spark for Scala 2.10 by following the instructions on the "Building Spark" page; however, it does mean that Scala 2.11 will be the default Scala version used by our CI builds (including pull request builds).

The Scala 2.11 compiler is faster than 2.10, so I think we'll be able to look forward to a slight speedup in our CI builds (it looks like it's about 2X faster for the Maven compile-only builds, for instance).

After this patch is merged, I'll update Jenkins to add new compile-only jobs to ensure that Scala 2.10 compilation doesn't break.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #10608 from JoshRosen/SPARK-6363.
2016-01-30 00:20:28 -08:00
Andrew Or 12252d1da9 [SPARK-13071] Coalescing HadoopRDD overwrites existing input metrics
This issue is causing tests to fail consistently in master with Hadoop 2.6 / 2.7. This is because for Hadoop 2.5+ we overwrite existing values of `InputMetrics#bytesRead` in each call to `HadoopRDD#compute`. In the case of coalesce, e.g.
```
sc.textFile(..., 4).coalesce(2).count()
```
we will call `compute` multiple times in the same task, overwriting `bytesRead` values from previous calls to `compute`.

For a regression test, see `InputOutputMetricsSuite.input metrics for old hadoop with coalesce`. I did not add a new regression test because it's impossible without significant refactoring; there's a lot of existing duplicate code in this corner of Spark.

This was caused by #10835.

Author: Andrew Or <andrew@databricks.com>

Closes #10973 from andrewor14/fix-input-metrics-coalesce.
2016-01-29 18:03:08 -08:00
Andrew Or 70e69fc4dd [SPARK-13088] Fix DAG viz in latest version of chrome
Apparently chrome removed `SVGElement.prototype.getTransformToElement`, which is used by our JS library dagre-d3 when creating edges. The real diff can be found here: 7d6c0002e4, which is taken from the fix in the main repo: 1ef067f1c6

Upstream issue: https://github.com/cpettitt/dagre-d3/issues/202

Author: Andrew Or <andrew@databricks.com>

Closes #10986 from andrewor14/fix-dag-viz.
2016-01-29 18:00:49 -08:00
Andrew Or e6ceac49a3 [SPARK-13096][TEST] Fix flaky verifyPeakExecutionMemorySet
Previously we would assert things before all events are guaranteed to have been processed. To fix this, just block until all events are actually processed, i.e. until the listener queue is empty.

https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7/79/testReport/junit/org.apache.spark.util.collection/ExternalAppendOnlyMapSuite/spilling/

Author: Andrew Or <andrew@databricks.com>

Closes #10990 from andrewor14/accum-suite-less-flaky.
2016-01-29 17:59:41 -08:00
Andrew Or e38b0baa38 [SPARK-13055] SQLHistoryListener throws ClassCastException
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.
2016-01-29 13:45:03 -08:00
zhuol e4c1162b6b [SPARK-10873] Support column sort and search for History Server.
[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.
2016-01-29 11:54:58 -06:00
Andrew Or d702f0c170 [HOTFIX] Fix Scala 2.11 compilation
by explicitly marking annotated parameters as vals (SI-8813).

Caused by #10835.

Author: Andrew Or <andrew@databricks.com>

Closes #10955 from andrewor14/fix-scala211.
2016-01-27 14:01:55 -08:00
Josh Rosen 32f741115b [SPARK-13021][CORE] Fail fast when custom RDDs violate RDD.partition's API contract
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.
2016-01-27 13:27:32 -08:00
Andrew Or 87abcf7df9 [SPARK-12895][SPARK-12896] Migrate TaskMetrics to accumulators
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.
2016-01-27 11:15:48 -08:00
Nishkam Ravi bae3c9a4eb [SPARK-12967][NETTY] Avoid NettyRpc error message during sparkContext shutdown
If there's an RPC issue while sparkContext is alive but stopped (which would happen only when executing SparkContext.stop), log a warning instead. This is a common occurrence.

vanzin

Author: Nishkam Ravi <nishkamravi@gmail.com>
Author: nishkamravi2 <nishkamravi@gmail.com>

Closes #10881 from nishkamravi2/master_netty.
2016-01-26 21:14:39 -08:00
Shixiong Zhu 22662b2416 [SPARK-12614][CORE] Don't throw non fatal exception from ask
Right now RpcEndpointRef.ask may throw exception in some corner cases, such as calling ask after stopping RpcEnv. It's better to avoid throwing exception from RpcEndpointRef.ask. We can send the exception to the future for `ask`.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #10568 from zsxwing/send-ask-fail.
2016-01-26 17:24:40 -08:00
zhuol ae0309a881 [SPARK-10911] Executors should System.exit on clean shutdown.
Call system.exit explicitly to make sure non-daemon user threads terminate. Without this, user applications might live forever if the cluster manager does not appropriately kill them. E.g., YARN had this bug: HADOOP-12441.

Author: zhuol <zhuol@yahoo-inc.com>

Closes #9946 from zhuoliu/10911.
2016-01-26 09:40:02 -06:00
Sean Owen 649e9d0f5b [SPARK-3369][CORE][STREAMING] Java mapPartitions Iterator->Iterable is inconsistent with Scala's Iterator->Iterator
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.
2016-01-26 11:55:28 +00:00
Liang-Chi Hsieh 5936bf9fa8 [SPARK-12961][CORE] Prevent snappy-java memory leak
JIRA: https://issues.apache.org/jira/browse/SPARK-12961

To prevent memory leak in snappy-java, just call the method once and cache the result. After the library releases new version, we can remove this object.

JoshRosen

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

Closes #10875 from viirya/prevent-snappy-memory-leak.
2016-01-26 11:36:00 +00:00
Davies Liu 7d877c3439 [SPARK-12902] [SQL] visualization for generated operators
This PR brings back visualization for generated operators, they looks like:

![sql](https://cloud.githubusercontent.com/assets/40902/12460920/0dc7956a-bf6b-11e5-9c3f-8389f452526e.png)

![stage](https://cloud.githubusercontent.com/assets/40902/12460923/11806ac4-bf6b-11e5-9c72-e84a62c5ea93.png)

Note: SQL metrics are not supported right now, because they are very slow, will be supported once we have batch mode.

Author: Davies Liu <davies@databricks.com>

Closes #10828 from davies/viz_codegen.
2016-01-25 12:44:20 -08:00
Alex Bozarth c037d25482 [SPARK-12149][WEB UI] Executor UI improvement suggestions - Color UI
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.
2016-01-25 14:42:44 -06:00
Michael Allman 4ee8191e57 [SPARK-12755][CORE] Stop the event logger before the DAG scheduler
[SPARK-12755][CORE] Stop the event logger before the DAG scheduler to avoid a race condition where the standalone master attempts to build the app's history UI before the event log is stopped.

This contribution is my original work, and I license this work to the Spark project under the project's open source license.

Author: Michael Allman <michael@videoamp.com>

Closes #10700 from mallman/stop_event_logger_first.
2016-01-25 09:51:41 +00:00
Shixiong Zhu ea5c38fe75 [HOTFIX]Remove rpcEnv.awaitTermination to avoid dead-lock in some test
Looks rpcEnv.awaitTermination may block some tests forever. Just remove it and investigate the tests.
2016-01-22 22:14:47 -08:00
Shixiong Zhu bc1babd63d [SPARK-7997][CORE] Remove Akka from Spark Core and Streaming
- 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.
2016-01-22 21:20:04 -08:00
Shixiong Zhu 944fdadf77 [SPARK-12847][CORE][STREAMING] Remove StreamingListenerBus and post all Streaming events to the same thread as Spark events
Including the following changes:

1. Add StreamingListenerForwardingBus to WrappedStreamingListenerEvent process events in `onOtherEvent` to StreamingListener
2. Remove StreamingListenerBus
3. Merge AsynchronousListenerBus and LiveListenerBus to the same class LiveListenerBus
4. Add `logEvent` method to SparkListenerEvent so that EventLoggingListener can use it to ignore WrappedStreamingListenerEvents

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #10779 from zsxwing/streaming-listener.
2016-01-20 11:57:53 -08:00
scwf 43f1d59e17 [SPARK-2750][WEB UI] Add https support to the Web UI
Author: scwf <wangfei1@huawei.com>
Author: Marcelo Vanzin <vanzin@cloudera.com>
Author: WangTaoTheTonic <wangtao111@huawei.com>
Author: w00228970 <wangfei1@huawei.com>

Closes #10238 from vanzin/SPARK-2750.
2016-01-19 14:49:55 -08:00
Andrew Or b122c861cd [SPARK-12887] Do not expose var's in TaskMetrics
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.
2016-01-19 10:58:51 -08:00
Andrew Or 2b5d11f34d [SPARK-12885][MINOR] Rename 3 fields in ShuffleWriteMetrics
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.
2016-01-18 19:22:29 -08:00
Josh Rosen b8cb548a43 [SPARK-10985][CORE] Avoid passing evicted blocks throughout BlockManager
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.
2016-01-18 13:34:12 -08:00
Andrew Or 302bb569f3 [SPARK-12884] Move classes to their own files for readability
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 moves classes to their own files to avoid having single monolithic ones that contain 10 different classes.

Parent PR: #10717

Author: Andrew Or <andrew@databricks.com>

Closes #10810 from andrewor14/move-things.
2016-01-18 13:27:18 -08:00