Commit graph

1608 commits

Author SHA1 Message Date
zsxwing 084e4e1262 [SPARK-6028][Core]A new RPC implemetation based on the network module
Design doc: https://docs.google.com/document/d/1CF5G6rGVQMKSyV_QKo4D2M-x6rxz5x1Ew7aK3Uq6u8c/edit?usp=sharing

Author: zsxwing <zsxwing@gmail.com>

Closes #6457 from zsxwing/new-rpc.
2015-09-23 18:59:49 -07:00
Tathagata Das 5548a25475 [SPARK-10652] [SPARK-10742] [STREAMING] Set meaningful job descriptions for all streaming jobs
Here is the screenshot after adding the job descriptions to threads that run receivers and the scheduler thread running the batch jobs.

## All jobs page
* Added job descriptions with links to relevant batch details page
![image](https://cloud.githubusercontent.com/assets/663212/9924165/cda4a372-5cb1-11e5-91ca-d43a32c699e9.png)

## All stages page
* Added stage descriptions with links to relevant batch details page
![image](https://cloud.githubusercontent.com/assets/663212/9923814/2cce266a-5cae-11e5-8a3f-dad84d06c50e.png)

## Streaming batch details page
* Added the +details link
![image](https://cloud.githubusercontent.com/assets/663212/9921977/24014a32-5c98-11e5-958e-457b6c38065b.png)

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #8791 from tdas/SPARK-10652.
2015-09-22 22:44:09 -07:00
Andrew Or 61d4c07f4b [SPARK-10640] History server fails to parse TaskCommitDenied
... simply because the code is missing!

Author: Andrew Or <andrew@databricks.com>

Closes #8828 from andrewor14/task-end-reason-json.
2015-09-22 16:35:43 -07:00
Josh Rosen 1ca5e2e0b8 [SPARK-10704] Rename HashShuffleReader to BlockStoreShuffleReader
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.
2015-09-22 11:50:22 -07:00
Tathagata Das 72869883f1 [SPARK-10649] [STREAMING] Prevent inheriting job group and irrelevant job description in streaming jobs
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.
2015-09-21 16:47:52 -07:00
hushan[胡珊] b78c65b03a [SPARK-5259] [CORE] don't submit stage until its dependencies map outputs are registered
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.
2015-09-21 14:26:15 -05:00
zsxwing ebbf85f07b [SPARK-7989] [SPARK-10651] [CORE] [TESTS] Increase timeout to fix flaky tests
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.
2015-09-21 11:39:04 -07:00
Josh Rosen 2117eea71e [SPARK-10710] Remove ability to disable spilling in core and SQL
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.
2015-09-19 21:40:21 -07:00
Reynold Xin 348d7c9a93 [SPARK-9808] Remove hash shuffle file consolidation.
Author: Reynold Xin <rxin@databricks.com>

Closes #8812 from rxin/SPARK-9808-1.
2015-09-18 13:48:41 -07:00
Jeff Zhang 36d8b278d8 [SPARK-10531] [CORE] AppId is set as AppName in status rest api
Verify it manually.

Author: Jeff Zhang <zjffdu@apache.org>

Closes #8688 from zjffdu/SPARK-10531.
2015-09-17 10:25:18 -07:00
Cheng Lian 69c9830d28 [MINOR] [CORE] Fixes minor variable name typo
Author: Cheng Lian <lian@databricks.com>

Closes #8784 from liancheng/typo-fix.
2015-09-17 00:48:57 -07:00
Josh Rosen 38700ea40c [SPARK-10381] Fix mixup of taskAttemptNumber & attemptId in OutputCommitCoordinator
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.
2015-09-15 17:11:21 -07:00
Andrew Or b6e998634e [SPARK-10548] [SPARK-10563] [SQL] Fix concurrent SQL executions
*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.
2015-09-15 16:45:47 -07:00
Matei Zaharia 1a0955250b [SPARK-9851] Support submitting map stages individually in DAGScheduler
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.
2015-09-14 21:47:40 -04:00
Andrew Or 7b6c856367 [SPARK-10564] ThreadingSuite: assertion failures in threads don't fail the test (round 2)
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.
2015-09-14 15:09:43 -07:00
Forest Fang fd1e8cddf2 [SPARK-10543] [CORE] Peak Execution Memory Quantile should be Per-task Basis
Read `PEAK_EXECUTION_MEMORY` using `update` to get per task partial value instead of cumulative value.

I tested with this workload:

```scala
val size = 1000
val repetitions = 10
val data = sc.parallelize(1 to size, 5).map(x => (util.Random.nextInt(size / repetitions),util.Random.nextDouble)).toDF("key", "value")
val res = data.toDF.groupBy("key").agg(sum("value")).count
```

Before:
![image](https://cloud.githubusercontent.com/assets/4317392/9828197/07dd6874-58b8-11e5-9bd9-6ba927c38b26.png)

After:
![image](https://cloud.githubusercontent.com/assets/4317392/9828151/a5ddff30-58b7-11e5-8d31-eda5dc4eae79.png)

Tasks view:
![image](https://cloud.githubusercontent.com/assets/4317392/9828199/17dc2b84-58b8-11e5-92a8-be89ce4d29d1.png)

cc andrewor14 I appreciate if you can give feedback on this since I think you introduced display of this metric.

Author: Forest Fang <forest.fang@outlook.com>

Closes #8726 from saurfang/stagepage.
2015-09-14 15:07:13 -07:00
Josh Rosen b3a7480ab0 [SPARK-10330] Add Scalastyle rule to require use of SparkHadoopUtil JobContext methods
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.
2015-09-12 16:23:55 -07:00
Sean Owen 22730ad54d [SPARK-10547] [TEST] Streamline / improve style of Java API tests
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.
2015-09-12 10:40:10 +01:00
Andrew Or d74c6a143c [SPARK-10564] ThreadingSuite: assertion failures in threads don't fail the test
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.
2015-09-11 15:02:59 -07:00
Matt Massie 0eabea8a05 [SPARK-9043] Serialize key, value and combiner classes in ShuffleDependency
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.
2015-09-10 17:24:33 -07:00
Iulian Dragos f0562e8cdb [SPARK-6350] [MESOS] Fine-grained mode scheduler respects mesosExecutor.cores
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.
2015-09-10 12:00:21 -07:00
mcheah af3bc59d1f [SPARK-8167] Make tasks that fail from YARN preemption not fail job
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.
2015-09-10 11:58:54 -07:00
Reynold Xin 5ffe752b59 [SPARK-9767] Remove ConnectionManager.
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.
2015-09-07 10:42:30 -10:00
robbins 2e1c17553d [SPARK-10454] [SPARK CORE] wait for empty event queue
Author: robbins <robbins@uk.ibm.com>

Closes #8605 from robbinspg/DAGSchedulerSuite-fix.
2015-09-04 15:23:29 -07:00
jeanlyn db4c130f9e [SPARK-9591] [CORE] Job may fail for exception during getting remote block
[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.
2015-09-03 13:56:11 -07:00
Vinod K C 11ef32c5a1 [SPARK-10430] [CORE] Added hashCode methods in AccumulableInfo and RDDOperationScope
Author: Vinod K C <vinod.kc@huawei.com>

Closes #8581 from vinodkc/fix_RDDOperationScope_Hashcode.
2015-09-03 13:55:02 -07:00
Pat Shields e62f4a46f4 [SPARK-9672] [MESOS] Don’t include SPARK_ENV_LOADED when passing env vars
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.
2015-09-03 13:53:18 -07:00
robbins d911c682f0 [SPARK-10431] [CORE] Fix intermittent test failure. Wait for event queue to be clear
Author: robbins <robbins@uk.ibm.com>

Closes #8582 from robbinspg/InputOutputMetricsSuite.
2015-09-03 13:47:25 -07:00
Davies Liu 62b4690d6b [SPARK-10379] preserve first page in UnsafeShuffleExternalSorter
Author: Davies Liu <davies@databricks.com>

Closes #8543 from davies/preserve_page.
2015-09-02 22:15:54 -07:00
Imran Rashid 3ddb9b3233 [SPARK-10247] [CORE] improve readability of a test case in DAGSchedulerSuite
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.
2015-09-02 22:14:50 -07:00
Ilya Ganelin 4bd85d06e0 [SPARK-5945] Spark should not retry a stage infinitely on a FetchFailedException
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.
2015-09-02 22:08:24 -07:00
zhuol ec01280533 [SPARK-4223] [CORE] Support * in acls.
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.
2015-09-01 11:14:59 -10:00
Davies Liu 540bdee931 [SPARK-10341] [SQL] fix memory starving in unsafe SMJ
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.
2015-08-31 15:55:22 -07:00
EugenCepoi 72f6dbf7b0 [SPARK-8730] Fixes - Deser objects containing a primitive class attribute
Author: EugenCepoi <cepoi.eugen@gmail.com>

Closes #7122 from EugenCepoi/master.
2015-08-31 13:24:35 -05:00
Marcelo Vanzin c53c902fa9 [SPARK-9284] [TESTS] Allow all tests to run without an assembly.
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.
2015-08-28 12:33:40 -07:00
Ram Sriharsha de0278286c [SPARK-10251] [CORE] some common types are not registered for Kryo Serializat…
…ion by default

Author: Ram Sriharsha <rsriharsha@hw11853.local>

Closes #8465 from harsha2010/SPARK-10251.
2015-08-26 23:12:55 -07:00
Sean Owen 69c9c17716 [SPARK-9613] [CORE] Ban use of JavaConversions and migrate all existing uses to JavaConverters
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.
2015-08-25 12:33:13 +01:00
Andrew Or 662bb96676 [SPARK-10144] [UI] Actually show peak execution memory by default
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.
2015-08-24 14:10:50 -07:00
Imran Rashid 708036c1de [SPARK-9439] [YARN] External shuffle service robust to NM restarts using leveldb
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
2015-08-21 08:41:36 -05:00
Marcelo Vanzin e0dd1309ac [SPARK-10119] [CORE] Fix isDynamicAllocationEnabled when config is expliticly disabled.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #8316 from vanzin/SPARK-10119.
2015-08-19 14:33:32 -07:00
Joshi f3391ff2b8 [SPARK-8889] [CORE] Fix for OOM for graph creation
Fix for OOM for graph creation

Author: Joshi <rekhajoshm@gmail.com>
Author: Rekha Joshi <rekhajoshm@gmail.com>

Closes #7602 from rekhajoshm/SPARK-8889.
2015-08-19 21:23:02 +01:00
Josh Rosen 010b03ed52 [SPARK-9952] Fix N^2 loop when DAGScheduler.getPreferredLocsInternal accesses cacheLocs
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.
2015-08-18 22:30:13 -07:00
Matei Zaharia cf016075a0 [SPARK-10008] Ensure shuffle locality doesn't take precedence over narrow deps
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.
2015-08-16 00:34:58 -07:00
Andrew Or 6518ef6303 [SPARK-9948] Fix flaky AccumulatorSuite - internal accumulators
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.
2015-08-14 13:42:53 -07:00
Andrew Or 8815ba2f67 [SPARK-9649] Fix MasterSuite, third time's a charm
This particular test did not load the default configurations so
it continued to start the REST server, which causes port bind
exceptions.
2015-08-13 11:31:10 -07:00
Josh Rosen 7b13ed27c1 [SPARK-9870] Disable driver UI and Master REST server in SparkSubmitSuite
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.
2015-08-12 18:52:11 -07:00
Niranjan Padmanabhan 738f353988 [SPARK-9092] Fixed incompatibility when both num-executors and dynamic...
… 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.
2015-08-12 16:10:21 -07:00
Andrew Or e0110792ef [SPARK-9747] [SQL] Avoid starving an unsafe operator in aggregation
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.
2015-08-12 10:08:35 -07:00
Andrew Or be5d191207 [SPARK-9795] Dynamic allocation: avoid double counting when killing same executor twice
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.
2015-08-12 09:24:50 -07:00
Tom White 2e680668f7 [SPARK-8625] [CORE] Propagate user exceptions in tasks back to driver
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.
2015-08-12 10:07:11 -05:00
xutingjun b85f9a242a [SPARK-8366] maxNumExecutorsNeeded should properly handle failed tasks
Author: xutingjun <xutingjun@huawei.com>
Author: meiyoula <1039320815@qq.com>

Closes #6817 from XuTingjun/SPARK-8366.
2015-08-11 23:19:35 -07:00
zsxwing f16bc68dfb [SPARK-9824] [CORE] Fix the issue that InternalAccumulator leaks WeakReference
`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.
2015-08-11 14:06:23 -07:00
Reynold Xin d378396f86 [SPARK-9815] Rename PlatformDependent.UNSAFE -> Platform.
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.
2015-08-11 08:41:06 -07:00
Carson Wang ef062c1599 [SPARK-9731] Standalone scheduling incorrect cores if spark.executor.cores is not set
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
2015-08-07 23:36:26 -07:00
Andrew Or 881548ab20 [SPARK-9674] Re-enable ignored test in SQLQuerySuite
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
2015-08-07 14:20:13 -07:00
Davies Liu 15bd6f338d [SPARK-9453] [SQL] support records larger than page size in UnsafeShuffleExternalSorter
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
2015-08-06 23:40:38 -07:00
Reynold Xin 4309262ec9 [SPARK-9700] Pick default page size more intelligently.
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.
2015-08-06 23:18:29 -07:00
Andrew Or 014a9f9d8c [SPARK-9709] [SQL] Avoid starving unsafe operators that use sort
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
2015-08-06 19:04:57 -07:00
Liang-Chi Hsieh 21fdfd7d6f [SPARK-9548][SQL] Add a destructive iterator for BytesToBytesMap
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.
2015-08-06 14:33:29 -07:00
Marcelo Vanzin 4399b7b090 [SPARK-9651] Fix UnsafeExternalSorterSuite.
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.
2015-08-05 17:58:36 -07:00
Andrew Or 5f0fb6466f [SPARK-9649] Fix flaky test MasterSuite - randomize ports
```
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
2015-08-05 14:12:22 -07:00
Takeshi YAMAMURO 6d8a6e4161 [SPARK-9360] [SQL] Support BinaryType in PrefixComparators for UnsafeExternalSort
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
2015-08-05 00:56:35 -07:00
Burak Yavuz c9a4c36d05 [SPARK-8313] R Spark packages support
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
2015-08-04 18:20:12 -07:00
CodingCat 9d668b7368 [SPARK-9602] remove "Akka/Actor" words from comments
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
2015-08-04 14:54:11 -07:00
Josh Rosen ab8ee1a3b9 [SPARK-9452] [SQL] Support records larger than page size in UnsafeExternalSorter
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.
2015-08-04 14:42:11 -07:00
Sean Owen 76d74090d6 [SPARK-9534] [BUILD] Enable javac lint for scalac parity; fix a lot of build warnings, 1.5.0 edition
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.
2015-08-04 12:02:26 +01:00
Burak Yavuz 1633d0a261 [SPARK-9263] Added flags to exclude dependencies when using --packages
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
2015-08-03 17:42:03 -07:00
Andrew Or 702aa9d7fb [SPARK-8735] [SQL] Expose memory usage for shuffles, joins and aggregations
This patch exposes the memory used by internal data structures on the SparkUI. This tracks memory used by all spilling operations and SQL operators backed by Tungsten, e.g. `BroadcastHashJoin`, `ExternalSort`, `GeneratedAggregate` etc. The metric exposed is "peak execution memory", which broadly refers to the peak in-memory sizes of each of these data structure.

A separate patch will extend this by linking the new information to the SQL operators themselves.

<img width="950" alt="screen shot 2015-07-29 at 7 43 17 pm" src="https://cloud.githubusercontent.com/assets/2133137/8974776/b90fc980-362a-11e5-9e2b-842da75b1641.png">
<img width="802" alt="screen shot 2015-07-29 at 7 43 05 pm" src="https://cloud.githubusercontent.com/assets/2133137/8974777/baa76492-362a-11e5-9b77-e364a6a6b64e.png">

<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/7770)
<!-- Reviewable:end -->

Author: Andrew Or <andrew@databricks.com>

Closes #7770 from andrewor14/expose-memory-metrics and squashes the following commits:

9abecb9 [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
f5b0d68 [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
d7df332 [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
8eefbc5 [Andrew Or] Fix non-failing tests
9de2a12 [Andrew Or] Fix tests due to another logical merge conflict
876bfa4 [Andrew Or] Fix failing test after logical merge conflict
361a359 [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
40b4802 [Andrew Or] Fix style?
d0fef87 [Andrew Or] Fix tests?
b3b92f6 [Andrew Or] Address comments
0625d73 [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
c00a197 [Andrew Or] Fix potential NPEs
10da1cd [Andrew Or] Fix compile
17f4c2d [Andrew Or] Fix compile?
a87b4d0 [Andrew Or] Fix compile?
d70874d [Andrew Or] Fix test compile + address comments
2840b7d [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
6aa2f7a [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
b889a68 [Andrew Or] Minor changes: comments, spacing, style
663a303 [Andrew Or] UnsafeShuffleWriter: update peak memory before close
d090a94 [Andrew Or] Fix style
2480d84 [Andrew Or] Expand test coverage
5f1235b [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
1ecf678 [Andrew Or] Minor changes: comments, style, unused imports
0b6926c [Andrew Or] Oops
111a05e [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
a7a39a5 [Andrew Or] Strengthen presence check for accumulator
a919eb7 [Andrew Or] Add tests for unsafe shuffle writer
23c845d [Andrew Or] Add tests for SQL operators
a757550 [Andrew Or] Address comments
b5c51c1 [Andrew Or] Re-enable test in JavaAPISuite
5107691 [Andrew Or] Add tests for internal accumulators
59231e4 [Andrew Or] Fix tests
9528d09 [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
5b5e6f3 [Andrew Or] Add peak execution memory to summary table + tooltip
92b4b6b [Andrew Or] Display peak execution memory on the UI
eee5437 [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
d9b9015 [Andrew Or] Track execution memory in unsafe shuffles
770ee54 [Andrew Or] Track execution memory in broadcast joins
9c605a4 [Andrew Or] Track execution memory in GeneratedAggregate
9e824f2 [Andrew Or] Add back execution memory tracking for *ExternalSort
4ef4cb1 [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
e6c3e2f [Andrew Or] Move internal accumulators creation to Stage
a417592 [Andrew Or] Expose memory metrics in UnsafeExternalSorter
3c4f042 [Andrew Or] Track memory usage in ExternalAppendOnlyMap / ExternalSorter
bd7ab3f [Andrew Or] Add internal accumulators to TaskContext
2015-08-03 14:22:07 -07:00
Andrew Or b41a32718d [SPARK-1855] Local checkpointing
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
2015-08-03 10:58:37 -07:00
Timothy Chen 95dccc6335 [SPARK-8873] [MESOS] Clean up shuffle files if external shuffle service is used
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.
2015-08-03 01:55:58 -07:00
Reynold Xin 2e981b7bfa [SPARK-9531] [SQL] UnsafeFixedWidthAggregationMap.destructAndCreateExternalSorter
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.
2015-08-02 12:32:14 -07:00
Andrew Or 6688ba6e68 [SPARK-4751] Dynamic allocation in standalone mode
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
2015-08-01 11:57:14 -07:00
Reynold Xin d90f2cf7a2 [SPARK-9517][SQL] BytesToBytesMap should encode data the same way as UnsafeExternalSorter
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.
2015-07-31 23:55:16 -07:00
Josh Rosen 8cb415a4b9 [SPARK-9451] [SQL] Support entries larger than default page size in BytesToBytesMap & integrate with ShuffleMemoryManager
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.
2015-07-31 19:19:27 -07:00
CodingCat c0686668ae [SPARK-9202] capping maximum number of executor&driver information kept in Worker
https://issues.apache.org/jira/browse/SPARK-9202

Author: CodingCat <zhunansjtu@gmail.com>

Closes #7714 from CodingCat/SPARK-9202 and squashes the following commits:

23977fb [CodingCat] add comments about why we don't synchronize finishedExecutors & finishedDrivers
dc9772d [CodingCat] addressing the comments
e125241 [CodingCat] stylistic fix
80bfe52 [CodingCat] fix JsonProtocolSuite
d7d9485 [CodingCat] styistic fix and respect insert ordering
031755f [CodingCat] add license info & stylistic fix
c3b5361 [CodingCat] test cases and docs
c557b3a [CodingCat] applications are fine
9cac751 [CodingCat] application is fine...
ad87ed7 [CodingCat] trimFinishedExecutorsAndDrivers
2015-07-31 20:27:00 +01:00
Reynold Xin e7a0976e99 [SPARK-9458][SPARK-9469][SQL] Code generate prefix computation in sorting & moves unsafe conversion out of TungstenSort.
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.
2015-07-30 17:17:27 -07:00
Reynold Xin 4a8bb9d00d Revert "[SPARK-9458] Avoid object allocation in prefix generation."
This reverts commit 9514d874f0.
2015-07-30 01:04:24 -07:00
Reynold Xin 07fd7d3647 [SPARK-9460] Avoid byte array allocation in StringPrefixComparator.
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.
2015-07-29 21:18:43 -07:00
Reynold Xin 9514d874f0 [SPARK-9458] Avoid object allocation in prefix generation.
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.
2015-07-29 20:46:03 -07:00
Josh Rosen 1b0099fc62 [SPARK-9411] [SQL] Make Tungsten page sizes configurable
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
2015-07-29 16:00:30 -07:00
Joseph Batchik 069a4c414d [SPARK-746] [CORE] Added Avro Serialization to Kryo
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.
2015-07-29 14:02:32 -05:00
Josh Rosen ea49705bd4 [SPARK-9419] ShuffleMemoryManager and MemoryStore should track memory on a per-task, not per-thread, basis
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
2015-07-28 21:53:28 -07:00
jerryshao ab62595661 [SPARK-4352] [YARN] [WIP] Incorporate locality preferences in dynamic allocation requests
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
2015-07-27 15:46:35 -07:00
Josh Rosen ecad9d4346 [SPARK-9364] Fix array out of bounds and use-after-free bugs in UnsafeExternalSorter
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
2015-07-27 09:34:49 -07:00
Andrew Or 1cf19760d6 [SPARK-9352] [SPARK-9353] Add tests for standalone scheduling code
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
2015-07-26 13:03:13 -07:00
Reynold Xin c8d71a4183 [SPARK-9305] Rename org.apache.spark.Row to Item.
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.
2015-07-24 09:38:13 -07:00
François Garillot 6cd28cc21e [SPARK-9236] [CORE] Make defaultPartitioner not reuse a parent RDD's partitioner if it has 0 partitions
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
2015-07-24 15:41:13 +01:00
Josh Rosen ac3ae0f2be [SPARK-9266] Prevent "managed memory leak detected" exception from masking original exception
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
2015-07-23 00:43:26 -07:00
Josh Rosen b217230f2a [SPARK-9144] Remove DAGScheduler.runLocallyWithinThread and spark.localExecution.enabled
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
2015-07-22 21:04:04 -07:00
Matei Zaharia fe26584a1f [SPARK-9244] Increase some memory defaults
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
2015-07-22 15:28:09 -07:00
zsxwing 4f7f1ee378 [SPARK-4598] [WEBUI] Task table pagination for the Stage page
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
2015-07-21 09:54:39 -07:00
Jacek Lewandowski 31954910d6 [SPARK-7171] Added a method to retrieve metrics sources in TaskContext
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
2015-07-21 09:53:33 -07:00
Liang-Chi Hsieh 9a4fd875b3 [SPARK-9128] [CORE] Get outerclasses and objects with only one method calling in ClosureCleaner
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.
2015-07-21 09:52:27 -07:00
Ben f67da43c39 [SPARK-9036] [CORE] SparkListenerExecutorMetricsUpdate messages not included in JsonProtocol
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
2015-07-21 09:51:13 -07:00
Kay Ousterhout 6364735bcc [SPARK-8875] Remove BlockStoreShuffleFetcher class
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
2015-07-21 01:12:51 -07:00
Josh Rosen c032b0bf92 [SPARK-8797] [SPARK-9146] [SPARK-9145] [SPARK-9147] Support NaN ordering and equality comparisons in Spark SQL
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.
2015-07-20 22:38:05 -07:00
Imran Rashid 80e2568b25 [SPARK-8103][core] DAGScheduler should not submit multiple concurrent attempts for a stage
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
2015-07-20 10:28:32 -07:00
Joshi 42d8a012f6 [SPARK-8593] [CORE] Sort app attempts by start time.
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
2015-07-17 22:47:28 +01:00
zsxwing 812b63bbee [SPARK-8857][SPARK-8859][Core]Add an internal flag to Accumulable and send internal accumulator updates to the driver via heartbeats
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
2015-07-16 21:09:09 -07:00
Andrew Or 96aa3340f4 [SPARK-8119] HeartbeatReceiver should replace executors, not kill
**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
2015-07-16 19:39:54 -07:00