Commit graph

4968 commits

Author SHA1 Message Date
Christian Kadner dc7e399fc0 [SPARK-11338] [WEBUI] Prepend app links on HistoryPage with uiRoot path
[SPARK-11338: HistoryPage not multi-tenancy enabled ...](https://issues.apache.org/jira/browse/SPARK-11338)
- `HistoryPage.scala` ...prepending all page links with the web proxy (`uiRoot`) path
- `HistoryServerSuite.scala` ...adding a test case to verify all site-relative links are prefixed when the environment variable `APPLICATION_WEB_PROXY_BASE` (or System property `spark.ui.proxyBase`) is set

Author: Christian Kadner <ckadner@us.ibm.com>

Closes #9291 from ckadner/SPARK-11338 and squashes the following commits:

01d2f35 [Christian Kadner] [SPARK-11338][WebUI] nit fixes
d054bd7 [Christian Kadner] [SPARK-11338][WebUI] prependBaseUri in method makePageLink
8bcb3dc [Christian Kadner] [SPARK-11338][WebUI] Prepend application links on HistoryPage with uiRoot path
2015-11-01 13:09:42 -08:00
Josh Rosen ac4118db2d [SPARK-11424] Guard against double-close() of RecordReaders
**TL;DR**: We can rule out one rare but potential cause of input stream corruption via defensive programming.

## Background

[MAPREDUCE-5918](https://issues.apache.org/jira/browse/MAPREDUCE-5918) is a bug where an instance of a decompressor ends up getting placed into a pool multiple times. Since the pool is backed by a list instead of a set, this can lead to the same decompressor being used in different places at the same time, which is not safe because those decompressors will overwrite each other's buffers. Sometimes this buffer sharing will lead to exceptions but other times it will might silently result in invalid / garbled input.

That Hadoop bug is fixed in Hadoop 2.7 but is still present in many Hadoop versions that we wish to support. As a result, I think that we should try to work around this issue in Spark via defensive programming to prevent RecordReaders from being closed multiple times.

So far, I've had a hard time coming up with explanations of exactly how double-`close()`s occur in practice, but I do have a couple of explanations that work on paper.

For instance, it looks like https://github.com/apache/spark/pull/7424, added in 1.5, introduces at least one extremely~rare corner-case path where Spark could double-close() a LineRecordReader instance in a way that triggers the bug. Here are the steps involved in the bad execution that I brainstormed up:

* [The task has finished reading input, so we call close()](https://github.com/apache/spark/blob/v1.5.1/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala#L168).
* [While handling the close call and trying to close the reader, reader.close() throws an exception]( https://github.com/apache/spark/blob/v1.5.1/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala#L190)
* We don't set `reader = null` after handling this exception, so the [TaskCompletionListener also ends up calling NewHadoopRDD.close()](https://github.com/apache/spark/blob/v1.5.1/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala#L156), which, in turn, closes the record reader again.

In this hypothetical situation, `LineRecordReader.close()` could [fail with an exception if its InputStream failed to close](https://github.com/apache/hadoop/blob/release-1.2.1/src/mapred/org/apache/hadoop/mapred/LineRecordReader.java#L212).
I googled for "Exception in RecordReader.close()" and it looks like it's possible for a closed Hadoop FileSystem to trigger an error there: [SPARK-757](https://issues.apache.org/jira/browse/SPARK-757), [SPARK-2491](https://issues.apache.org/jira/browse/SPARK-2491)

Looking at [SPARK-3052](https://issues.apache.org/jira/browse/SPARK-3052), it seems like it's possible to get spurious exceptions there when there is an error reading from Hadoop. If the Hadoop FileSystem were to get into an error state _right_ after reading the last record then it looks like we could hit the bug here in 1.5.

## The fix

This patch guards against these issues by modifying `HadoopRDD.close()` and `NewHadoopRDD.close()` so that they set `reader = null` even if an exception occurs in the `reader.close()` call. In addition, I modified `NextIterator. closeIfNeeded()` to guard against double-close if the first `close()` call throws an exception.

I don't have an easy way to test this, since I haven't been able to reproduce the bug that prompted this patch, but these changes seem safe and seem to rule out the on-paper reproductions that I was able to brainstorm up.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9382 from JoshRosen/hadoop-decompressor-pooling-fix and squashes the following commits:

5ec97d7 [Josh Rosen] Add SqlNewHadoopRDD.unsetInputFileName() that I accidentally deleted.
ae46cf4 [Josh Rosen] Merge remote-tracking branch 'origin/master' into hadoop-decompressor-pooling-fix
087aa63 [Josh Rosen] Guard against double-close() of RecordReaders.
2015-10-31 10:47:22 -07:00
Davies Liu 45029bfdea [SPARK-11423] remove MapPartitionsWithPreparationRDD
Since we do not need to preserve a page before calling compute(), MapPartitionsWithPreparationRDD is not needed anymore.

This PR basically revert #8543, #8511, #8038, #8011

Author: Davies Liu <davies@databricks.com>

Closes #9381 from davies/remove_prepare2.
2015-10-30 15:47:40 -07:00
Sun Rui fab710a917 [SPARK-11414][SPARKR] Forgot to update usage of 'spark.sparkr.r.command' in RRDD in the PR for SPARK-10971.
Author: Sun Rui <rui.sun@intel.com>

Closes #9368 from sun-rui/SPARK-11414.
2015-10-30 10:51:11 -07:00
Iulian Dragos 0451b00148 [SPARK-10986][MESOS] Set the context class loader in the Mesos executor backend.
See [SPARK-10986](https://issues.apache.org/jira/browse/SPARK-10986) for details.

This fixes the `ClassNotFoundException` for Spark classes in the serializer.

I am not sure this is the right way to handle the class loader, but I couldn't find any documentation on how the context class loader is used and who relies on it. It seems at least the serializer uses it to instantiate classes during deserialization.

I am open to suggestions (I tried this fix on a real Mesos cluster and it *does* fix the issue).

tnachen andrewor14

Author: Iulian Dragos <jaguarul@gmail.com>

Closes #9282 from dragos/issue/mesos-classloader.
2015-10-30 16:51:32 +00:00
Davies Liu 56419cf11f [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPARK-10929] [SQL] Cooperative memory management
This PR introduce a mechanism to call spill() on those SQL operators that support spilling (for example, BytesToBytesMap, UnsafeExternalSorter and ShuffleExternalSorter) if there is not enough memory for execution. The preserved first page is needed anymore, so removed.

Other Spillable objects in Spark core (ExternalSorter and AppendOnlyMap) are not included in this PR, but those could benefit from this (trigger others' spilling).

The PrepareRDD may be not needed anymore, could be removed in follow up PR.

The following script will fail with OOM before this PR, finished in 150 seconds with 2G heap (also works in 1.5 branch, with similar duration).

```python
sqlContext.setConf("spark.sql.shuffle.partitions", "1")
df = sqlContext.range(1<<25).selectExpr("id", "repeat(id, 2) as s")
df2 = df.select(df.id.alias('id2'), df.s.alias('s2'))
j = df.join(df2, df.id==df2.id2).groupBy(df.id).max("id", "id2")
j.explain()
print j.count()
```

For thread-safety, here what I'm got:

1) Without calling spill(), the operators should only be used by single thread, no safety problems.

2) spill() could be triggered in two cases, triggered by itself, or by other operators. we can check trigger == this in spill(), so it's still in the same thread, so safety problems.

3) if it's triggered by other operators (right now cache will not trigger spill()), we only spill the data into disk when it's in scanning stage (building is finished), so the in-memory sorter or memory pages are read-only, we only need to synchronize the iterator and change it.

4) During scanning, the iterator will only use one record in one page, we can't free this page, because the downstream is currently using it (used by UnsafeRow or other objects). In BytesToBytesMap, we just skip the current page, and dump all others into disk. In UnsafeExternalSorter, we keep the page that is used by current record (having the same baseObject), free it when loading the next record. In ShuffleExternalSorter, the spill() will not trigger during scanning.

5) In order to avoid deadlock, we didn't call acquireMemory during spill (so we reused the pointer array in InMemorySorter).

Author: Davies Liu <davies@databricks.com>

Closes #9241 from davies/force_spill.
2015-10-29 23:38:06 -07:00
Kay Ousterhout b960a89056 [SPARK-11178] Improving naming around task failures.
Commit af3bc59d1f introduced new
functionality so that if an executor dies for a reason that's not
caused by one of the tasks running on the executor (e.g., due to
pre-emption), Spark doesn't count the failure towards the maximum
number of failures for the task.  That commit introduced some vague
naming that this commit attempts to fix; in particular:

(1) The variable "isNormalExit", which was used to refer to cases where
the executor died for a reason unrelated to the tasks running on the
machine, has been renamed (and reversed) to "exitCausedByApp". The problem
with the existing name is that it's not clear (at least to me!) what it
means for an exit to be "normal"; the new name is intended to make the
purpose of this variable more clear.

(2) The variable "shouldEventuallyFailJob" has been renamed to
"countTowardsTaskFailures". This variable is used to determine whether
a task's failure should be counted towards the maximum number of failures
allowed for a task before the associated Stage is aborted. The problem
with the existing name is that it can be confused with implying that
the task's failure should immediately cause the stage to fail because it
is somehow fatal (this is the case for a fetch failure, for example: if
a task fails because of a fetch failure, there's no point in retrying,
and the whole stage should be failed).

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #9164 from kayousterhout/SPARK-11178.
2015-10-27 16:55:10 -07:00
zsxwing 9fbd75ab5d [SPARK-11212][CORE][STREAMING] Make preferred locations support ExecutorCacheTaskLocation and update…
… ReceiverTracker and ReceiverSchedulingPolicy to use it

This PR includes the following changes:

1. Add a new preferred location format, `executor_<host>_<executorID>` (e.g., "executor_localhost_2"), to support specifying the executor locations for RDD.
2. Use the new preferred location format in `ReceiverTracker` to optimize the starting time of Receivers when there are multiple executors in a host.

The goal of this PR is to enable the streaming scheduler to place receivers (which run as tasks) in specific executors. Basically, I want to have more control on the placement of the receivers such that they are evenly distributed among the executors. We tried to do this without changing the core scheduling logic. But it does not allow specifying particular executor as preferred location, only at the host level. So if there are two executors in the same host, and I want two receivers to run on them (one on each executor), I cannot specify that. Current code only specifies the host as preference, which may end up launching both receivers on the same executor. We try to work around it but restarting a receiver when it does not launch in the desired executor and hope that next time it will be started in the right one. But that cause lots of restarts, and delays in correctly launching the receiver.

So this change, would allow the streaming scheduler to specify the exact executor as the preferred location. Also this is not exposed to the user, only the streaming scheduler uses this.

Author: zsxwing <zsxwing@gmail.com>

Closes #9181 from zsxwing/executor-location.
2015-10-27 16:14:33 -07:00
Kay Ousterhout 9fc16a82ad [SPARK-11306] Fix hang when JVM exits.
This commit fixes a bug where, in Standalone mode, if a task fails and crashes the JVM, the
failure is considered a "normal failure" (meaning it's considered unrelated to the task), so
the failure isn't counted against the task's maximum number of failures:
af3bc59d1f (diff-a755f3d892ff2506a7aa7db52022d77cL138).
As a result, if a task fails in a way that results in it crashing the JVM, it will continuously be
re-launched, resulting in a hang. This commit fixes that problem.

This bug was introduced by #8007; andrewor14 mccheah vanzin can you take a look at this?

This error is hard to trigger because we handle executor losses through 2 code paths (the second is via Akka, where Akka notices that the executor endpoint is disconnected).  In my setup, the Akka code path completes first, and doesn't have this bug, so things work fine (see my recent email to the dev list about this).  If I manually disable the Akka code path, I can see the hang (and this commit fixes the issue).

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #9273 from kayousterhout/SPARK-11306.
2015-10-27 10:46:43 -07:00
Sem Mulder feb8d6a44f [SPARK-11276][CORE] SizeEstimator prevents class unloading
The SizeEstimator keeps a cache of ClassInfos but this cache uses Class objects as keys.
Which results in strong references to the Class objects. If these classes are dynamically created
this prevents the corresponding ClassLoader from being GCed. Leading to PermGen exhaustion.

We use a Map with WeakKeys to prevent this issue.

Author: Sem Mulder <sem.mulder@site2mobile.com>

Closes #9244 from SemMulder/fix-sizeestimator-classunloading.
2015-10-27 07:55:10 +00:00
Sun Rui dc3220ce11 [SPARK-11209][SPARKR] Add window functions into SparkR [step 1].
Author: Sun Rui <rui.sun@intel.com>

Closes #9193 from sun-rui/SPARK-11209.
2015-10-26 20:58:18 -07:00
Kevin Yu 616be29c7f [SPARK-5966][WIP] Spark-submit deploy-mode cluster is not compatible with master local>
… master local>

Author: Kevin Yu <qyu@us.ibm.com>

Closes #9220 from kevinyu98/working_on_spark-5966.
2015-10-26 09:35:19 +00:00
Josh Rosen 85e654c5ec [SPARK-10984] Simplify *MemoryManager class structure
This patch refactors the MemoryManager class structure. After #9000, Spark had the following classes:

- MemoryManager
- StaticMemoryManager
- ExecutorMemoryManager
- TaskMemoryManager
- ShuffleMemoryManager

This is fairly confusing. To simplify things, this patch consolidates several of these classes:

- ShuffleMemoryManager and ExecutorMemoryManager were merged into MemoryManager.
- TaskMemoryManager is moved into Spark Core.

**Key changes and tasks**:

- [x] Merge ExecutorMemoryManager into MemoryManager.
  - [x] Move pooling logic into Allocator.
- [x] Move TaskMemoryManager from `spark-unsafe` to `spark-core`.
- [x] Refactor the existing Tungsten TaskMemoryManager interactions so Tungsten code use only this and not both this and ShuffleMemoryManager.
- [x] Refactor non-Tungsten code to use the TaskMemoryManager instead of ShuffleMemoryManager.
- [x] Merge ShuffleMemoryManager into MemoryManager.
  - [x] Move code
  - [x] ~~Simplify 1/n calculation.~~ **Will defer to followup, since this needs more work.**
- [x] Port ShuffleMemoryManagerSuite tests.
- [x] Move classes from `unsafe` package to `memory` package.
- [ ] Figure out how to handle the hacky use of the memory managers in HashedRelation's broadcast variable construction.
- [x] Test porting and cleanup: several tests relied on mock functionality (such as `TestShuffleMemoryManager.markAsOutOfMemory`) which has been changed or broken during the memory manager consolidation
  - [x] AbstractBytesToBytesMapSuite
  - [x] UnsafeExternalSorterSuite
  - [x] UnsafeFixedWidthAggregationMapSuite
  - [x] UnsafeKVExternalSorterSuite

**Compatiblity notes**:

- This patch introduces breaking changes in `ExternalAppendOnlyMap`, which is marked as `DevloperAPI` (likely for legacy reasons): this class now cannot be used outside of a task.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9127 from JoshRosen/SPARK-10984.
2015-10-25 21:19:52 -07:00
Bryan Cutler 80279ac187 [SPARK-11287] Fixed class name to properly start TestExecutor from deploy.client.TestClient
Executing deploy.client.TestClient fails due to bad class name for TestExecutor in ApplicationDescription.

Author: Bryan Cutler <bjcutler@us.ibm.com>

Closes #9255 from BryanCutler/fix-TestClient-classname-SPARK-11287.
2015-10-25 19:05:45 +00:00
Jacek Laskowski 146da0d810 Fix typos
Two typos squashed.

BTW Let me know how to proceed with other typos if I ran across any. I don't feel well to leave them aside as much as sending pull requests with such tiny changes. Guide me.

Author: Jacek Laskowski <jacek.laskowski@deepsense.io>

Closes #9250 from jaceklaskowski/typos-hunting.
2015-10-25 01:33:22 +01:00
Jeff Zhang ffed00493a [SPARK-11125] [SQL] Uninformative exception when running spark-sql witho…
…ut building with -Phive-thriftserver and SPARK_PREPEND_CLASSES is set

This is the exception after this patch. Please help review.
```
java.lang.NoClassDefFoundError: org/apache/hadoop/hive/cli/CliDriver
	at java.lang.ClassLoader.defineClass1(Native Method)
	at java.lang.ClassLoader.defineClass(ClassLoader.java:800)
	at java.security.SecureClassLoader.defineClass(SecureClassLoader.java:142)
	at java.net.URLClassLoader.defineClass(URLClassLoader.java:449)
	at java.net.URLClassLoader.access$100(URLClassLoader.java:71)
	at java.net.URLClassLoader$1.run(URLClassLoader.java:361)
	at java.net.URLClassLoader$1.run(URLClassLoader.java:355)
	at java.security.AccessController.doPrivileged(Native Method)
	at java.net.URLClassLoader.findClass(URLClassLoader.java:354)
	at java.lang.ClassLoader.loadClass(ClassLoader.java:425)
	at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308)
	at java.lang.ClassLoader.loadClass(ClassLoader.java:412)
	at java.lang.ClassLoader.loadClass(ClassLoader.java:358)
	at java.lang.Class.forName0(Native Method)
	at java.lang.Class.forName(Class.java:270)
	at org.apache.spark.util.Utils$.classForName(Utils.scala:173)
	at org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:647)
	at org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:180)
	at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:205)
	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:120)
	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
Caused by: java.lang.ClassNotFoundException: org.apache.hadoop.hive.cli.CliDriver
	at java.net.URLClassLoader$1.run(URLClassLoader.java:366)
	at java.net.URLClassLoader$1.run(URLClassLoader.java:355)
	at java.security.AccessController.doPrivileged(Native Method)
	at java.net.URLClassLoader.findClass(URLClassLoader.java:354)
	at java.lang.ClassLoader.loadClass(ClassLoader.java:425)
	at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308)
	at java.lang.ClassLoader.loadClass(ClassLoader.java:358)
	... 21 more
Failed to load hive class.
You need to build Spark with -Phive and -Phive-thriftserver.
```

Author: Jeff Zhang <zjffdu@apache.org>

Closes #9134 from zjffdu/SPARK-11125.
2015-10-23 22:56:55 -07:00
Sun Rui 2462dbcce8 [SPARK-10971][SPARKR] RRunner should allow setting path to Rscript.
Add a new spark conf option "spark.sparkr.r.driver.command" to specify the executable for an R script in client modes.

The existing spark conf option "spark.sparkr.r.command" is used to specify the executable for an R script in cluster modes for both driver and workers. See also [launch R worker script](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/api/r/RRDD.scala#L395).

BTW, [envrionment variable "SPARKR_DRIVER_R"](https://github.com/apache/spark/blob/master/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java#L275) is used to locate R shell on the local host.

For your information, PYSPARK has two environment variables serving simliar purpose:
PYSPARK_PYTHON	      Python binary executable to use for PySpark in both driver and workers (default is `python`).
PYSPARK_DRIVER_PYTHON	Python binary executable to use for PySpark in driver only (default is PYSPARK_PYTHON).
pySpark use the code [here](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala#L41) to determine the python executable for a python script.

Author: Sun Rui <rui.sun@intel.com>

Closes #9179 from sun-rui/SPARK-10971.
2015-10-23 21:38:04 -07:00
Jacek Laskowski b1c1597e3c Fix a (very tiny) typo
Author: Jacek Laskowski <jacek.laskowski@deepsense.io>

Closes #9230 from jaceklaskowski/utils-seconds-typo.
2015-10-22 22:42:15 -07:00
Marcelo Vanzin fa6a4fbf08 [SPARK-11134][CORE] Increase LauncherBackendSuite timeout.
This test can take a little while to finish on slow / loaded machines.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #9235 from vanzin/SPARK-11134.
2015-10-22 22:41:21 -07:00
zsxwing a88c66ca87 [SPARK-11098][CORE] Add Outbox to cache the sending messages to resolve the message disorder issue
The current NettyRpc has a message order issue because it uses a thread pool to send messages. E.g., running the following two lines in the same thread,

```
ref.send("A")
ref.send("B")
```

The remote endpoint may see "B" before "A" because sending "A" and "B" are in parallel.
To resolve this issue, this PR added an outbox for each connection, and if we are connecting to the remote node when sending messages, just cache the sending messages in the outbox and send them one by one when the connection is established.

Author: zsxwing <zsxwing@gmail.com>

Closes #9197 from zsxwing/rpc-outbox.
2015-10-22 21:01:01 -07:00
Andrew Or 34e71c6d89 [SPARK-11251] Fix page size calculation in local mode
```
// My machine only has 8 cores
$ bin/spark-shell --master local[32]
scala> val df = sc.parallelize(Seq((1, 1), (2, 2))).toDF("a", "b")
scala> df.as("x").join(df.as("y"), $"x.a" === $"y.a").count()

Caused by: java.io.IOException: Unable to acquire 2097152 bytes of memory
	at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPage(UnsafeExternalSorter.java:351)
```

Author: Andrew Or <andrew@databricks.com>

Closes #9209 from andrewor14/fix-local-page-size.
2015-10-22 15:58:08 -07:00
Kay Ousterhout 3535b91ddc [SPARK-11163] Remove unnecessary addPendingTask calls.
This commit removes unnecessary calls to addPendingTask in
TaskSetManager.executorLost. These calls are unnecessary: for
tasks that are still pending and haven't been launched, they're
still in all of the correct pending lists, so calling addPendingTask
has no effect. For tasks that are currently running (which may still be
in the pending lists, depending on how they were scheduled), we call
addPendingTask in handleFailedTask, so the calls at the beginning
of executorLost are redundant.

I think these calls are left over from when we re-computed the locality
levels in addPendingTask; now that we call recomputeLocality separately,
I don't think these are necessary.

Now that those calls are removed, the readding parameter in addPendingTask
is no longer necessary, so this commit also removes that parameter.

markhamstra can you take a look at this?

cc vanzin

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #9154 from kayousterhout/SPARK-11163.
2015-10-22 11:39:06 -07:00
zsxwing 7bb6d31cff [SPARK-11232][CORE] Use 'offer' instead of 'put' to make sure calling send won't be interrupted
The current `NettyRpcEndpointRef.send` can be interrupted because it uses `LinkedBlockingQueue.put`, which may hang the application.

Image the following execution order:

  | thread 1: TaskRunner.kill | thread 2: TaskRunner.run
------------- | ------------- | -------------
1 | killed = true |
2 |  | if (killed) {
3 |  | throw new TaskKilledException
4 |  | case _: TaskKilledException  _: InterruptedException if task.killed =>
5 | task.kill(interruptThread): interruptThread is true |
6 | | execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled))
7 | | localEndpoint.send(StatusUpdate(taskId, state, serializedData)): in LocalBackend

Then `localEndpoint.send(StatusUpdate(taskId, state, serializedData))` will throw `InterruptedException`. This will prevent the executor from updating the task status and hang the application.

An failure caused by the above issue here: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44062/consoleFull

Since `receivers` is an unbounded `LinkedBlockingQueue`, we can just use `LinkedBlockingQueue.offer` to resolve this issue.

Author: zsxwing <zsxwing@gmail.com>

Closes #9198 from zsxwing/dont-interrupt-send.
2015-10-22 11:31:47 -07:00
Josh Rosen f6d06adf05 [SPARK-10708] Consolidate sort shuffle implementations
There's a lot of duplication between SortShuffleManager and UnsafeShuffleManager. Given that these now provide the same set of functionality, now that UnsafeShuffleManager supports large records, I think that we should replace SortShuffleManager's serialized shuffle implementation with UnsafeShuffleManager's and should merge the two managers together.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #8829 from JoshRosen/consolidate-sort-shuffle-implementations.
2015-10-22 09:46:30 -07:00
zhichao.li c03b6d1158 [SPARK-11121][CORE] Correct the TaskLocation type
Correct the logic to return `HDFSCacheTaskLocation` instance when the input `str` is a in memory location.

Author: zhichao.li <zhichao.li@intel.com>

Closes #9096 from zhichao-li/uselessBranch.
2015-10-22 03:59:26 -07:00
Reynold Xin 555b2086a1 Minor cleanup of ShuffleMapStage.outputLocs code.
I was looking at this code and found the documentation to be insufficient. I added more documentation, and refactored some relevant code path slightly to improve encapsulation. There are more that I want to do, but I want to get these changes in before doing more work.

My goal is to reduce exposing internal fields directly in ShuffleMapStage to improve encapsulation. After this change, DAGScheduler no longer directly writes outputLocs. There are still 3 places that reads outputLocs directly, but we can change those later.

Author: Reynold Xin <rxin@databricks.com>

Closes #9175 from rxin/stage-cleanup.
2015-10-21 15:33:13 -07:00
Holden Karau e18b571c33 [SPARK-10447][SPARK-3842][PYSPARK] upgrade pyspark to py4j0.9
Upgrade to Py4j0.9

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

Closes #8615 from holdenk/SPARK-10447-upgrade-pyspark-to-py4j0.9.
2015-10-20 10:52:49 -07:00
Jakob Odersky 60851bc7bf [SPARK-11110][BUILD] Remove transient annotation for parameters.
`transient` annotations on class parameters (not case class parameters or vals) causes compilation errors during compilation with Scala 2.11.
I understand that transient *parameters* make no sense, however I don't quite understand why the 2.10 compiler accepted them.

Note: in case it is preferred to keep the annotations in case someone would in the future want to redefine them as vals, it would also be possible to just add `val` after the annotation, e.g. `class Foo(transient x: Int)` becomes `class Foo(transient private val x: Int)`.

I chose to remove the annotation as it also reduces needles clutter, however please feel free to tell me if you prefer the second option and I'll update the PR

Author: Jakob Odersky <jodersky@gmail.com>

Closes #9126 from jodersky/sbt-scala-2.11.
2015-10-20 08:54:34 +01:00
Jean-Baptiste Onofré 8f74aa6397 [SPARK-10876] Display total uptime for completed applications
Author: Jean-Baptiste Onofré <jbonofre@apache.org>

Closes #9059 from jbonofre/SPARK-10876.
2015-10-20 08:45:39 +01:00
Ryan Williams 16906ef23a [SPARK-11120] Allow sane default number of executor failures when dynamically allocating in YARN
I also added some information to container-failure error msgs about what host they failed on, which would have helped me identify the problem that lead me to this JIRA and PR sooner.

Author: Ryan Williams <ryan.blake.williams@gmail.com>

Closes #9147 from ryan-williams/dyn-exec-failures.
2015-10-19 16:34:15 -07:00
Chris Bannister fc26f32cf1 [SPARK-9708][MESOS] Spark should create local temporary directories in Mesos sandbox when launched with Mesos
This is my own original work and I license this to the project under the project's open source license

Author: Chris Bannister <chris.bannister@swiftkey.com>
Author: Chris Bannister <chris.bannister@swiftkey.net>

Closes #8358 from Zariel/mesos-local-dir.
2015-10-19 16:24:40 -07:00
Liang-Chi Hsieh a1413b3662 [SPARK-11051][CORE] Do not allow local checkpointing after the RDD is materialized and checkpointed
JIRA: https://issues.apache.org/jira/browse/SPARK-11051

When a `RDD` is materialized and checkpointed, its partitions and dependencies are cleared. If we allow local checkpointing on it and assign `LocalRDDCheckpointData` to its `checkpointData`. Next time when the RDD is materialized again, the error will be thrown.

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

Closes #9072 from viirya/no-localcheckpoint-after-checkpoint.
2015-10-19 16:16:31 -07:00
Marcelo Vanzin 7ab0ce6501 [SPARK-11131][CORE] Fix race in worker registration protocol.
Because the registration RPC was not really an RPC, but a bunch of
disconnected messages, it was possible for other messages to be
sent before the reply to the registration arrived, and that would
confuse the Worker. Especially in local-cluster mode, the worker was
succeptible to receiving an executor request before it received a
message from the master saying registration succeeded.

On top of the above, the change also fixes a ClassCastException when
the registration fails, which also affects the executor registration
protocol. Because the `ask` is issued with a specific return type,
if the error message (of a different type) was returned instead, the
code would just die with an exception. This is fixed by having a common
base trait for these reply messages.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #9138 from vanzin/SPARK-11131.
2015-10-19 16:14:50 -07:00
Jacek Laskowski bd64c2d550 [SPARK-10921][YARN] Completely remove the use of SparkContext.prefer…
…redNodeLocationData

Author: Jacek Laskowski <jacek.laskowski@deepsense.io>

Closes #8976 from jaceklaskowski/SPARK-10921.
2015-10-19 09:59:18 +01:00
ph 022a8f6a1f [SPARK-11129] [MESOS] Link Spark WebUI from Mesos WebUI
Mesos has a feature for linking to frameworks running on top of Mesos
from the Mesos WebUI. This commit enables Spark to make use of this
feature so one can directly visit the running Spark WebUIs from the
Mesos WebUI.

Author: ph <ph@plista.com>

Closes #9135 from philipphoffmann/SPARK-11129.
2015-10-17 15:37:51 -07:00
Reynold Xin 2549374206 [SPARK-11165] Logging trait should be private - not DeveloperApi.
Its classdoc actually says; "NOTE: DO NOT USE this class outside of Spark. It is intended as an internal utility."

Author: Reynold Xin <rxin@databricks.com>

Closes #9155 from rxin/private-logging-trait.
2015-10-17 12:41:42 -07:00
gweidner ac09a3a465 [SPARK-11109] [CORE] Move FsHistoryProvider off deprecated AccessControlException
Switched from deprecated org.apache.hadoop.fs.permission.AccessControlException to org.apache.hadoop.security.AccessControlException.

Author: gweidner <gweidner@us.ibm.com>

Closes #9144 from gweidner/SPARK-11109.
2015-10-16 14:02:12 -07:00
navis.ryu b9c5e5d4ac [SPARK-11124] JsonParser/Generator should be closed for resource recycle
Some json parsers are not closed. parser in JacksonParser#parseJson, for example.

Author: navis.ryu <navis@apache.org>

Closes #9130 from navis/SPARK-11124.
2015-10-16 11:19:37 -07:00
Andrew Or 3b364ff0a4 [SPARK-11078] Ensure spilling tests actually spill
#9084 uncovered that many tests that test spilling don't actually spill. This is a follow-up patch to fix that to ensure our unit tests actually catch potential bugs in spilling. The size of this patch is inflated by the refactoring of `ExternalSorterSuite`, which had a lot of duplicate code and logic.

Author: Andrew Or <andrew@databricks.com>

Closes #9124 from andrewor14/spilling-tests.
2015-10-15 14:50:01 -07:00
KaiXinXiaoLei 2d000124b7 [SPARK-10515] When killing executor, the pending replacement executors should not be lost
If the heartbeat receiver kills executors (and new ones are not registered to replace them), the idle timeout for the old executors will be lost (and then change a total number of executors requested by Driver), So new ones will be not to asked to replace them.
For example, executorsPendingToRemove=Set(1), and executor 2 is idle timeout before a new executor is asked to replace executor 1. Then driver kill executor 2, and sending RequestExecutors to AM. But executorsPendingToRemove=Set(1,2), So AM doesn't allocate a executor to replace 1.

see: https://github.com/apache/spark/pull/8668

Author: KaiXinXiaoLei <huleilei1@huawei.com>
Author: huleilei <huleilei1@huawei.com>

Closes #8945 from KaiXinXiaoLei/pendingexecutor.
2015-10-15 14:48:01 -07:00
Carson Wang d45a0d3ca2 [SPARK-11047] Internal accumulators miss the internal flag when replaying events in the history server
Internal accumulators don't write the internal flag to event log. So on the history server Web UI, all accumulators are not internal. This causes incorrect peak execution memory and unwanted accumulator table displayed on the stage page.
To fix it, I add the "internal" property of AccumulableInfo when writing the event log.

Author: Carson Wang <carson.wang@intel.com>

Closes #9061 from carsonwang/accumulableBug.
2015-10-15 10:36:54 -07:00
shellberg 523adc24a6 [SPARK-11066] Update DAGScheduler's "misbehaved ResultHandler"
Restrict tasks (of job) to only 1 to ensure that the causing Exception asserted for job failure is the deliberately thrown DAGSchedulerSuiteDummyException intended, not an UnsupportedOperationException from any second/subsequent tasks that can propagate from a race condition during code execution.

Author: shellberg <sah@zepler.org>

Closes #9076 from shellberg/shellberg-DAGSchedulerSuite-misbehavedResultHandlerTest-patch-1.
2015-10-15 18:07:10 +01:00
Adam Lewandowski 0f62c2282b [SPARK-11093] [CORE] ChildFirstURLClassLoader#getResources should return all found resources, not just those in the child classloader
Author: Adam Lewandowski <alewandowski@ipcoop.com>

Closes #9106 from alewando/childFirstFix.
2015-10-15 09:45:54 -07:00
Reynold Xin cf2e0ae720 [SPARK-11096] Post-hoc review Netty based RPC implementation - round 2
A few more changes:

1. Renamed IDVerifier -> RpcEndpointVerifier
2. Renamed NettyRpcAddress -> RpcEndpointAddress
3. Simplified NettyRpcHandler a bit by removing the connection count tracking. This is OK because I now force spark.shuffle.io.numConnectionsPerPeer to 1
4. Reduced spark.rpc.connect.threads to 64. It would be great to eventually remove this extra thread pool.
5. Minor cleanup & documentation.

Author: Reynold Xin <rxin@databricks.com>

Closes #9112 from rxin/SPARK-11096.
2015-10-14 12:41:02 -07:00
Tom Graves 135a2ce5b0 [SPARK-10619] Can't sort columns on Executor Page
should pick into spark 1.5.2 also.

https://issues.apache.org/jira/browse/SPARK-10619

looks like this was broken by commit: fb1d06fc24 (diff-b8adb646ef90f616c34eb5c98d1ebd16)
It looks like somethings were change to use the UIUtils.listingTable but executor page wasn't converted so when it removed sortable from the UIUtils. TABLE_CLASS_NOT_STRIPED it broke this page.

Simply add the sortable tag back in and it fixes both active UI and the history server UI.

Author: Tom Graves <tgraves@yahoo-inc.com>

Closes #9101 from tgravescs/SPARK-10619.
2015-10-14 10:12:25 -07:00
Andrew Or b3ffac5178 [SPARK-10983] Unified memory manager
This patch unifies the memory management of the storage and execution regions such that either side can borrow memory from each other. When memory pressure arises, storage will be evicted in favor of execution. To avoid regressions in cases where storage is crucial, we dynamically allocate a fraction of space for storage that execution cannot evict. Several configurations are introduced:

- **spark.memory.fraction (default 0.75)**: ​fraction of the heap space used for execution and storage. The lower this is, the more frequently spills and cached data eviction occur. The purpose of this config is to set aside memory for internal metadata, user data structures, and imprecise size estimation in the case of sparse, unusually large records.

- **spark.memory.storageFraction (default 0.5)**: size of the storage region within the space set aside by `s​park.memory.fraction`. ​Cached data may only be evicted if total storage exceeds this region.

- **spark.memory.useLegacyMode (default false)**: whether to use the memory management that existed in Spark 1.5 and before. This is mainly for backward compatibility.

For a detailed description of the design, see [SPARK-10000](https://issues.apache.org/jira/browse/SPARK-10000). This patch builds on top of the `MemoryManager` interface introduced in #9000.

Author: Andrew Or <andrew@databricks.com>

Closes #9084 from andrewor14/unified-memory-manager.
2015-10-13 13:49:59 -07:00
Sun Rui 5e3868ba13 [SPARK-10051] [SPARKR] Support collecting data of StructType in DataFrame
Two points in this PR:

1.    Originally thought was that a named R list is assumed to be a struct in SerDe. But this is problematic because some R functions will implicitly generate named lists that are not intended to be a struct when transferred by SerDe. So SerDe clients have to explicitly mark a names list as struct by changing its class from "list" to "struct".

2.    SerDe is in the Spark Core module, and data of StructType is represented as GenricRow which is defined in Spark SQL module. SerDe can't import GenricRow as in maven build  Spark SQL module depends on Spark Core module. So this PR adds a registration hook in SerDe to allow SQLUtils in Spark SQL module to register its functions for serialization and deserialization of StructType.

Author: Sun Rui <rui.sun@intel.com>

Closes #8794 from sun-rui/SPARK-10051.
2015-10-13 10:02:21 -07:00
Reynold Xin 1797055dbf [SPARK-11079] Post-hoc review Netty-based RPC - round 1
I'm going through the implementation right now for post-doc review. Adding more comments and renaming things as I go through them.

I also want to write higher level documentation about how the whole thing works -- but those will come in other pull requests.

Author: Reynold Xin <rxin@databricks.com>

Closes #9091 from rxin/rpc-review.
2015-10-13 09:51:20 -07:00
Tom Graves 63c340a710 [SPARK-10858] YARN: archives/jar/files rename with # doesn't work unl
https://issues.apache.org/jira/browse/SPARK-10858

The issue here is that in resolveURI we default to calling new File(path).getAbsoluteFile().toURI().  But if the path passed in already has a # in it then File(path) will think that is supposed to be part of the actual file path and not a fragment so it changes # to %23. Then when we try to parse that  later in Client as a URI it doesn't recognize there is a fragment.

so to fix we just check if there is a fragment, still create the File like we did before and then add the fragment back on.

Author: Tom Graves <tgraves@yahoo-inc.com>

Closes #9035 from tgravescs/SPARK-10858.
2015-10-09 14:06:25 -07:00
Marcelo Vanzin 015f7ef503 [SPARK-8673] [LAUNCHER] API and infrastructure for communicating with child apps.
This change adds an API that encapsulates information about an app
launched using the library. It also creates a socket-based communication
layer for apps that are launched as child processes; the launching
application listens for connections from launched apps, and once
communication is established, the channel can be used to send updates
to the launching app, or to send commands to the child app.

The change also includes hooks for local, standalone/client and yarn
masters.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #7052 from vanzin/SPARK-8673.
2015-10-09 15:28:09 -05:00
Andrew Or 67fbecbf32 [SPARK-10956] Common MemoryManager interface for storage and execution
This patch introduces a `MemoryManager` that is the central arbiter of how much memory to grant to storage and execution. This patch is primarily concerned only with refactoring while preserving the existing behavior as much as possible.

This is the first step away from the existing rigid separation of storage and execution memory, which has several major drawbacks discussed on the [issue](https://issues.apache.org/jira/browse/SPARK-10956). It is the precursor of a series of patches that will attempt to address those drawbacks.

Author: Andrew Or <andrew@databricks.com>
Author: Josh Rosen <joshrosen@databricks.com>
Author: andrewor14 <andrew@databricks.com>

Closes #9000 from andrewor14/memory-manager.
2015-10-08 21:44:59 -07:00
Marcelo Vanzin 56a9692fc0 [SPARK-10987] [YARN] Workaround for missing netty rpc disconnection event.
In YARN client mode, when the AM connects to the driver, it may be the case
that the driver never needs to send a message back to the AM (i.e., no
dynamic allocation or preemption). This triggers an issue in the netty rpc
backend where no disconnection event is sent to endpoints, and the AM never
exits after the driver shuts down.

The real fix is too complicated, so this is a quick hack to unblock YARN
client mode until we can work on the real fix. It forces the driver to
send a message to the AM when the AM registers, thus establishing that
connection and enabling the disconnection event when the driver goes
away.

Also, a minor side issue: when the executor is shutting down, it needs
to send an "ack" back to the driver when using the netty rpc backend; but
that "ack" wasn't being sent because the handler was shutting down the rpc
env before returning. So added a change to delay the shutdown a little bit,
allowing the ack to be sent back.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #9021 from vanzin/SPARK-10987.
2015-10-08 09:47:58 -07:00
Marcelo Vanzin 6ca27f8550 [SPARK-10964] [YARN] Correctly register the AM with the driver.
The `self` method returns null when called from the constructor;
instead, registration should happen in the `onStart` method, at
which point the `self` reference has already been initialized.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #9005 from vanzin/SPARK-10964.
2015-10-07 11:38:47 -07:00
Marcelo Vanzin 744f03e700 [SPARK-10916] [YARN] Set perm gen size when launching containers on YARN.
This makes YARN containers behave like all other processes launched by
Spark, which launch with a default perm gen size of 256m unless
overridden by the user (or not needed by the vm).

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #8970 from vanzin/SPARK-10916.
2015-10-06 10:17:12 -07:00
Reynold Xin ae6570ec2b Remove TODO in ShuffleMemoryManager. 2015-10-03 18:08:25 -07:00
zsxwing 107320c9bb [SPARK-6028] [CORE] Remerge #6457: new RPC implemetation and also pick #8905
This PR just reverted 02144d6745 to remerge #6457 and also included the commits in #8905.

Author: zsxwing <zsxwing@gmail.com>

Closes #8944 from zsxwing/SPARK-6028.
2015-10-03 01:04:35 -07:00
Joshi f85aa06464 [SPARK-10317] [CORE] Compatibility between history server script and functionality
Compatibility between history server script and functionality

The history server has its argument parsing class in HistoryServerArguments. However, this doesn't get involved in the start-history-server.sh codepath where the $0 arg is assigned to spark.history.fs.logDirectory and all other arguments discarded (e.g --property-file.)
This stops the other options being usable from this script

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

Closes #8758 from rekhajoshm/SPARK-10317.
2015-10-02 15:26:11 -07:00
Takeshi YAMAMURO 2272962eb0 [SPARK-9867] [SQL] Move utilities for binary data into ByteArray
The utilities such as Substring#substringBinarySQL and BinaryPrefixComparator#computePrefix for binary data are put together in ByteArray for easy-to-read.

Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes #8122 from maropu/CleanUpForBinaryType.
2015-10-01 21:33:27 -04:00
zsxwing 9b3e7768a2 [SPARK-10058] [CORE] [TESTS] Fix the flaky tests in HeartbeatReceiverSuite
Fixed the test failure here: https://amplab.cs.berkeley.edu/jenkins/view/Spark-QA-Test/job/Spark-1.5-SBT/116/AMPLAB_JENKINS_BUILD_PROFILE=hadoop2.2,label=spark-test/testReport/junit/org.apache.spark/HeartbeatReceiverSuite/normal_heartbeat/

This failure is because `HeartbeatReceiverSuite. heartbeatReceiver` may receive `SparkListenerExecutorAdded("driver")` sent from [LocalBackend](8fb3a65cbb/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala (L121)).

There are other race conditions in `HeartbeatReceiverSuite` because `HeartbeatReceiver.onExecutorAdded` and `HeartbeatReceiver.onExecutorRemoved` are asynchronous. This PR also fixed them.

Author: zsxwing <zsxwing@gmail.com>

Closes #8946 from zsxwing/SPARK-10058.
2015-10-01 07:09:31 -07:00
Sun Rui c7b29ae641 [SPARK-10851] [SPARKR] Exception not failing R applications (in yarn cluster mode)
The YARN backend doesn't like when user code calls System.exit, since it cannot know the exit status and thus cannot set an appropriate final status for the application.

This PR remove the usage of system.exit to exit the RRunner. Instead, when the R process running an SparkR script returns an exit code other than 0, throws SparkUserAppException which will be caught by ApplicationMaster and ApplicationMaster knows it failed. For other failures, throws SparkException.

Author: Sun Rui <rui.sun@intel.com>

Closes #8938 from sun-rui/SPARK-10851.
2015-09-30 11:03:08 -07:00
zsxwing dba95ea032 [SPARK-10825] [CORE] [TESTS] Fix race conditions in StandaloneDynamicAllocationSuite
Fix the following issues in StandaloneDynamicAllocationSuite:

1. It should not assume master and workers start in order
2. It should not assume master and workers get ready at once
3. It should not assume the application is already registered with master after creating SparkContext
4. It should not access Master.app and idToApp which are not thread safe

The changes includes:
* Use `eventually` to wait until master and workers are ready to fix 1 and 2
* Use `eventually`  to wait until the application is registered with master to fix 3
* Use `askWithRetry[MasterStateResponse](RequestMasterState)` to get the application info to fix 4

Author: zsxwing <zsxwing@gmail.com>

Closes #8914 from zsxwing/fix-StandaloneDynamicAllocationSuite.
2015-09-29 11:53:28 -07:00
Sean Owen bf4199e261 [SPARK-10833] [BUILD] Inline, organize BSD/MIT licenses in LICENSE
In the course of https://issues.apache.org/jira/browse/LEGAL-226 it came to light that the guidance at http://www.apache.org/dev/licensing-howto.html#permissive-deps means that permissively-licensed dependencies has a different interpretation than we (er, I) had been operating under. "pointer ... to the license within the source tree" specifically means a copy of the license within Spark's distribution, whereas at the moment, Spark's LICENSE has a pointer to the project's license in the other project's source tree.

The remedy is simply to inline all such license references (i.e. BSD/MIT licenses) or include their text in "licenses" subdirectory and point to that.

Along the way, we can also treat other BSD/MIT licenses, whose text has been inlined into LICENSE, in the same way.

The LICENSE file can continue to provide a helpful list of BSD/MIT licensed projects and a pointer to their sites. This would be over and above including license text in the distro, which is the essential thing.

Author: Sean Owen <sowen@cloudera.com>

Closes #8919 from srowen/SPARK-10833.
2015-09-28 22:56:43 -04:00
Holden Karau d8d50ed388 [SPARK-10812] [YARN] Spark hadoop util support switching to yarn
While this is likely not a huge issue for real production systems, for test systems which may setup a Spark Context and tear it down and stand up a Spark Context with a different master (e.g. some local mode & some yarn mode) tests this cane be an issue. Discovered during work on spark-testing-base on Spark 1.4.1, but seems like the logic that triggers it is present in master (see SparkHadoopUtil object). A valid work around for users encountering this issue is to fork a different JVM, however this can be heavy weight.

```
[info] SampleMiniClusterTest:
[info] Exception encountered when attempting to run a suite with class name: com.holdenkarau.spark.testing.SampleMiniClusterTest *** ABORTED ***
[info] java.lang.ClassCastException: org.apache.spark.deploy.SparkHadoopUtil cannot be cast to org.apache.spark.deploy.yarn.YarnSparkHadoopUtil
[info] at org.apache.spark.deploy.yarn.YarnSparkHadoopUtil$.get(YarnSparkHadoopUtil.scala:163)
[info] at org.apache.spark.deploy.yarn.Client.prepareLocalResources(Client.scala:257)
[info] at org.apache.spark.deploy.yarn.Client.createContainerLaunchContext(Client.scala:561)
[info] at org.apache.spark.deploy.yarn.Client.submitApplication(Client.scala:115)
[info] at org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend.start(YarnClientSchedulerBackend.scala:57)
[info] at org.apache.spark.scheduler.TaskSchedulerImpl.start(TaskSchedulerImpl.scala:141)
[info] at org.apache.spark.SparkContext.<init>(SparkContext.scala:497)
[info] at com.holdenkarau.spark.testing.SharedMiniCluster$class.setup(SharedMiniCluster.scala:186)
[info] at com.holdenkarau.spark.testing.SampleMiniClusterTest.setup(SampleMiniClusterTest.scala:26)
[info] at com.holdenkarau.spark.testing.SharedMiniCluster$class.beforeAll(SharedMiniCluster.scala:103)
```

Author: Holden Karau <holden@pigscanfly.ca>

Closes #8911 from holdenk/SPARK-10812-spark-hadoop-util-support-switching-to-yarn.
2015-09-28 06:33:45 -07:00
Matei Zaharia 21fd12cb17 [SPARK-9852] Let reduce tasks fetch multiple map output partitions
This makes two changes:

- Allow reduce tasks to fetch multiple map output partitions -- this is a pretty small change to HashShuffleFetcher
- Move shuffle locality computation out of DAGScheduler and into ShuffledRDD / MapOutputTracker; this was needed because the code in DAGScheduler wouldn't work for RDDs that fetch multiple map output partitions from each reduce task

I also added an AdaptiveSchedulingSuite that creates RDDs depending on multiple map output partitions.

Author: Matei Zaharia <matei@databricks.com>

Closes #8844 from mateiz/spark-9852.
2015-09-24 23:39:04 -04:00
Josh Rosen 8023242e77 [SPARK-10761] Refactor DiskBlockObjectWriter to not require BlockId
The DiskBlockObjectWriter constructor took a BlockId parameter but never used it. As part of some general cleanup in these interfaces, this patch refactors its constructor to eliminate this parameter.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #8871 from JoshRosen/disk-block-object-writer-blockid-cleanup.
2015-09-24 14:18:33 -07:00
Xiangrui Meng 02144d6745 Revert "[SPARK-6028][Core]A new RPC implemetation based on the network module"
This reverts commit 084e4e1262.
2015-09-24 08:25:44 -07:00
Andrew Or 83f6f54d12 [SPARK-10474] [SQL] Aggregation fails to allocate memory for pointer array (round 2)
This patch reverts most of the changes in a previous fix #8827.

The real cause of the issue is that in `TungstenAggregate`'s prepare method we only reserve 1 page, but later when we switch to sort-based aggregation we try to acquire 1 page AND a pointer array. The longer-term fix should be to reserve also the pointer array, but for now ***we will simply not track the pointer array***. (Note that elsewhere we already don't track the pointer array, e.g. [here](a18208047f/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java (L88)))

Note: This patch reuses the unit test added in #8827 so it doesn't show up in the diff.

Author: Andrew Or <andrew@databricks.com>

Closes #8888 from andrewor14/dont-track-pointer-array.
2015-09-23 19:34:31 -07:00
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
Reynold Xin 9952217749 [SPARK-10731] [SQL] Delegate to Scala's DataFrame.take implementation in Python DataFrame.
Python DataFrame.head/take now requires scanning all the partitions. This pull request changes them to delegate the actual implementation to Scala DataFrame (by calling DataFrame.take).

This is more of a hack for fixing this issue in 1.5.1. A more proper fix is to change executeCollect and executeTake to return InternalRow rather than Row, and thus eliminate the extra round-trip conversion.

Author: Reynold Xin <rxin@databricks.com>

Closes #8876 from rxin/SPARK-10731.
2015-09-23 16:43:21 -07:00
tedyu 27bfa9ab3a [SPARK-10721] Log warning when file deletion fails
Author: tedyu <yuzhihong@gmail.com>

Closes #8843 from tedyu/master.
2015-09-23 10:01:28 +01: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
Reynold Xin a96ba40f7e [SPARK-10714] [SPARK-8632] [SPARK-10685] [SQL] Refactor Python UDF handling
This patch refactors Python UDF handling:

1. Extract the per-partition Python UDF calling logic from PythonRDD into a PythonRunner. PythonRunner itself expects iterator as input/output, and thus has no dependency on RDD. This way, we can use PythonRunner directly in a mapPartitions call, or in the future in an environment without RDDs.
2. Use PythonRunner in Spark SQL's BatchPythonEvaluation.
3. Updated BatchPythonEvaluation to only use its input once, rather than twice. This should fix Python UDF performance regression in Spark 1.5.

There are a number of small cleanups I wanted to do when I looked at the code, but I kept most of those out so the diff looks small.

This basically implements the approach in https://github.com/apache/spark/pull/8833, but with some code moving around so the correctness doesn't depend on the inner workings of Spark serialization and task execution.

Author: Reynold Xin <rxin@databricks.com>

Closes #8835 from rxin/python-iter-refactor.
2015-09-22 14:11:46 -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
xutingjun 2ea0f2e11b [SPARK-9585] Delete the input format caching because some input format are non thread safe
If we cache the  InputFormat, all tasks on the same executor will share it.
Some InputFormat is thread safety, but some are not, such as HiveHBaseTableInputFormat. If tasks share a non thread safe InputFormat, unexpected error may be occurs.
To avoid it, I think we should delete the input format  caching.

Author: xutingjun <xutingjun@huawei.com>
Author: meiyoula <1039320815@qq.com>
Author: Xutingjun <xutingjun@huawei.com>

Closes #7918 from XuTingjun/cached_inputFormat.
2015-09-22 11:01:32 -07:00
Rekha Joshi 7278f792a7 [SPARK-10718] [BUILD] Update License on conf files and corresponding excludes file update
Update License on conf files and corresponding excludes file update

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

Closes #8842 from rekhajoshm/SPARK-10718.
2015-09-22 11:03:21 +01:00
Andrew Or fd61b00487 [Minor] style fix for previous commit f24316e 2015-09-22 00:05:30 -07:00
Madhusudanan Kandasamy f24316e6d9 [SPARK-10458] [SPARK CORE] Added isStopped() method in SparkContext
Added isStopped() method in SparkContext

Author: Madhusudanan Kandasamy <madhusudanan@in.ibm.com>

Closes #8749 from kmadhugit/SPARK-10458.
2015-09-22 00:03:48 -07:00
Hossein c986e933a9 [SPARK-10711] [SPARKR] Do not assume spark.submit.deployMode is always set
In ```RUtils.sparkRPackagePath()``` we
1. Call ``` sys.props("spark.submit.deployMode")``` which returns null if ```spark.submit.deployMode``` is not suet
2. Call ``` sparkConf.get("spark.submit.deployMode")``` which throws ```NoSuchElementException``` if ```spark.submit.deployMode``` is not set. This patch simply passes a default value ("cluster") for ```spark.submit.deployMode```.

cc rxin

Author: Hossein <hossein@databricks.com>

Closes #8832 from falaki/SPARK-10711.
2015-09-21 21:09:59 -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
Andrew Or 7ff8d68cc1 [SPARK-10474] [SQL] Aggregation fails to allocate memory for pointer array
When `TungstenAggregation` hits memory pressure, it switches from hash-based to sort-based aggregation in-place. However, in the process we try to allocate the pointer array for writing to the new `UnsafeExternalSorter` *before* actually freeing the memory from the hash map. This lead to the following exception:
```
 java.io.IOException: Could not acquire 65536 bytes of memory
        at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.initializeForWriting(UnsafeExternalSorter.java:169)
        at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.spill(UnsafeExternalSorter.java:220)
        at org.apache.spark.sql.execution.UnsafeKVExternalSorter.<init>(UnsafeKVExternalSorter.java:126)
        at org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap.destructAndCreateExternalSorter(UnsafeFixedWidthAggregationMap.java:257)
        at org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.switchToSortBasedAggregation(TungstenAggregationIterator.scala:435)
```

Author: Andrew Or <andrew@databricks.com>

Closes #8827 from andrewor14/allocate-pointer-array.
2015-09-18 23:58:25 -07:00
Mingyu Kim 8074208fa4 [SPARK-10611] Clone Configuration for each task for NewHadoopRDD
This patch attempts to fix the Hadoop Configuration thread safety issue for NewHadoopRDD in the same way SPARK-2546 fixed the issue for HadoopRDD.

Author: Mingyu Kim <mkim@palantir.com>

Closes #8763 from mingyukim/mkim/SPARK-10611.
2015-09-18 15:40:58 -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
linweizhong 93c7650ab6 [SPARK-9522] [SQL] SparkSubmit process can not exit if kill application when HiveThriftServer was starting
When we start HiveThriftServer, we will start SparkContext first, then start HiveServer2, if we kill application while HiveServer2 is starting then SparkContext will stop successfully, but SparkSubmit process can not exit.

Author: linweizhong <linweizhong@huawei.com>

Closes #7853 from Sephiroth-Lin/SPARK-9522.
2015-09-17 22:25:24 -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
Josiah Samuel 81b4db374d [SPARK-10172] [CORE] disable sort in HistoryServer webUI
This pull request is to address the JIRA SPARK-10172 (History Server web UI gets messed up when sorting on any column).
The content of the table gets messed up due to the rowspan attribute of the table data(cell) during sorting.
The current table sort library used in SparkUI (sorttable.js) doesn't support/handle cells(td) with rowspans.
The fix will disable the table sort in the web UI, when there are jobs listed with multiple attempts.

Author: Josiah Samuel <josiah_sams@in.ibm.com>

Closes #8506 from josiahsams/SPARK-10172.
2015-09-17 10:18:21 -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
Sun Rui 896edb51ab [SPARK-10050] [SPARKR] Support collecting data of MapType in DataFrame.
1. Support collecting data of MapType from DataFrame.
2. Support data of MapType in createDataFrame.

Author: Sun Rui <rui.sun@intel.com>

Closes #8711 from sun-rui/SPARK-10050.
2015-09-16 13:20:39 -07:00
Sean Owen 5dbaf3d391 [SPARK-10589] [WEBUI] Add defense against external site framing
Set `X-Frame-Options: SAMEORIGIN` to protect against frame-related vulnerability

Author: Sean Owen <sowen@cloudera.com>

Closes #8745 from srowen/SPARK-10589.
2015-09-16 19:19:23 +01: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
vinodkc 99ecfa5945 [SPARK-10575] [SPARK CORE] Wrapped RDD.takeSample with Scope
Remove return statements in RDD.takeSample and wrap it withScope

Author: vinodkc <vinod.kc.in@gmail.com>
Author: vinodkc <vinodkc@users.noreply.github.com>
Author: Vinod K C <vinod.kc@huawei.com>

Closes #8730 from vinodkc/fix_takesample_return.
2015-09-15 17:01:39 -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
Reynold Xin 09b7e7c198 Update version to 1.6.0-SNAPSHOT.
Author: Reynold Xin <rxin@databricks.com>

Closes #8350 from rxin/1.6.
2015-09-15 00:54:20 -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
Sean Owen 4e2242bb41 [SPARK-10576] [BUILD] Move .java files out of src/main/scala
Move .java files in `src/main/scala` to `src/main/java` root, except for `package-info.java` (to stay next to package.scala)

Author: Sean Owen <sowen@cloudera.com>

Closes #8736 from srowen/SPARK-10576.
2015-09-14 15:03:51 -07:00
Wenchen Fan 32407bfd2b [SPARK-9899] [SQL] log warning for direct output committer with speculation enabled
This is a follow-up of https://github.com/apache/spark/pull/8317.

When speculation is enabled, there may be multiply tasks writing to the same path. Generally it's OK as we will write to a temporary directory first and only one task can commit the temporary directory to target path.

However, when we use direct output committer, tasks will write data to target path directly without temporary directory. This causes problems like corrupted data. Please see [PR comment](https://github.com/apache/spark/pull/8191#issuecomment-131598385) for more details.

Unfortunately, we don't have a simple flag to tell if a output committer will write to temporary directory or not, so for safety, we have to disable any customized output committer when `speculation` is true.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8687 from cloud-fan/direct-committer.
2015-09-14 11:51:39 -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
Nithin Asokan 8285e3b0d3 [SPARK-10554] [CORE] Fix NPE with ShutdownHook
https://issues.apache.org/jira/browse/SPARK-10554

Fixes NPE when ShutdownHook tries to cleanup temporary folders

Author: Nithin Asokan <Nithin.Asokan@Cerner.com>

Closes #8720 from nasokan/SPARK-10554.
2015-09-12 09:50:49 +01:00
Daniel Imfeld 6d8367807c [SPARK-10566] [CORE] SnappyCompressionCodec init exception handling masks important error information
When throwing an IllegalArgumentException in SnappyCompressionCodec.init, chain the existing exception. This allows potentially important debugging info to be passed to the user.

Manual testing shows the exception chained properly, and the test suite still looks fine as well.

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

Author: Daniel Imfeld <daniel@danielimfeld.com>

Closes #8725 from dimfeld/dimfeld-patch-1.
2015-09-12 09:19:59 +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
tedyu b231ab8938 [SPARK-10546] Check partitionId's range in ExternalSorter#spill()
See this thread for background:
http://search-hadoop.com/m/q3RTt0rWvIkHAE81

We should check the range of partition Id and provide meaningful message through exception.

Alternatively, we can use abs() and modulo to force the partition Id into legitimate range. However, expectation is that user should correct the logic error in his / her code.

Author: tedyu <yuzhihong@gmail.com>

Closes #8703 from tedyu/master.
2015-09-11 21:45:45 +01: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
Sun Rui 45e3be5c13 [SPARK-10049] [SPARKR] Support collecting data of ArraryType in DataFrame.
this PR :
1.  Enhance reflection in RBackend. Automatically matching a Java array to Scala Seq when finding methods. Util functions like seq(), listToSeq() in R side can be removed, as they will conflict with the Serde logic that transferrs a Scala seq to R side.

2.  Enhance the SerDe to support transferring  a Scala seq to R side. Data of ArrayType in DataFrame
after collection is observed to be of Scala Seq type.

3.  Support ArrayType in createDataFrame().

Author: Sun Rui <rui.sun@intel.com>

Closes #8458 from sun-rui/SPARK-10049.
2015-09-10 12:21:13 -07:00
Akash Mishra a5ef2d0600 [SPARK-10514] [MESOS] waiting for min no of total cores acquired by Spark by implementing the sufficientResourcesRegistered method
spark.scheduler.minRegisteredResourcesRatio configuration parameter works for YARN mode but not for Mesos Coarse grained mode.

If the parameter specified default value of 0 will be set for spark.scheduler.minRegisteredResourcesRatio in base class and this method will always return true.

There are no existing test for YARN mode too. Hence not added test for the same.

Author: Akash Mishra <akash.mishra20@gmail.com>

Closes #8672 from SleepyThread/master.
2015-09-10 12:04:02 -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
Cheng Hao e048111376 [SPARK-10466] [SQL] UnsafeRow SerDe exception with data spill
Data Spill with UnsafeRow causes assert failure.

```
java.lang.AssertionError: assertion failed
	at scala.Predef$.assert(Predef.scala:165)
	at org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$2.writeKey(UnsafeRowSerializer.scala:75)
	at org.apache.spark.storage.DiskBlockObjectWriter.write(DiskBlockObjectWriter.scala:180)
	at org.apache.spark.util.collection.ExternalSorter$$anonfun$writePartitionedFile$2$$anonfun$apply$1.apply(ExternalSorter.scala:688)
	at org.apache.spark.util.collection.ExternalSorter$$anonfun$writePartitionedFile$2$$anonfun$apply$1.apply(ExternalSorter.scala:687)
	at scala.collection.Iterator$class.foreach(Iterator.scala:727)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
	at org.apache.spark.util.collection.ExternalSorter$$anonfun$writePartitionedFile$2.apply(ExternalSorter.scala:687)
	at org.apache.spark.util.collection.ExternalSorter$$anonfun$writePartitionedFile$2.apply(ExternalSorter.scala:683)
	at scala.collection.Iterator$class.foreach(Iterator.scala:727)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
	at org.apache.spark.util.collection.ExternalSorter.writePartitionedFile(ExternalSorter.scala:683)
	at org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:80)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
	at org.apache.spark.scheduler.Task.run(Task.scala:88)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
```

To reproduce that with code (thanks andrewor14):
```scala
bin/spark-shell --master local
  --conf spark.shuffle.memoryFraction=0.005
  --conf spark.shuffle.sort.bypassMergeThreshold=0

sc.parallelize(1 to 2 * 1000 * 1000, 10)
  .map { i => (i, i) }.toDF("a", "b").groupBy("b").avg().count()
```

Author: Cheng Hao <hao.cheng@intel.com>

Closes #8635 from chenghao-intel/unsafe_spill.
2015-09-10 11:48:43 -07:00
Liang-Chi Hsieh 45de518742 [SPARK-9730] [SQL] Add Full Outer Join support for SortMergeJoin
This PR is based on #8383 , thanks to viirya

JIRA: https://issues.apache.org/jira/browse/SPARK-9730

This patch adds the Full Outer Join support for SortMergeJoin. A new class SortMergeFullJoinScanner is added to scan rows from left and right iterators. FullOuterIterator is simply a wrapper of type RowIterator to consume joined rows from SortMergeFullJoinScanner.

Closes #8383

Author: Liang-Chi Hsieh <viirya@appier.com>
Author: Davies Liu <davies@databricks.com>

Closes #8579 from davies/smj_fullouter.
2015-09-09 16:02:27 -07:00
Luc Bourlier c1bc4f439f [SPARK-10227] fatal warnings with sbt on Scala 2.11
The bulk of the changes are on `transient` annotation on class parameter. Often the compiler doesn't generate a field for this parameters, so the the transient annotation would be unnecessary.
But if the class parameter are used in methods, then fields are created. So it is safer to keep the annotations.

The remainder are some potential bugs, and deprecated syntax.

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

Closes #8433 from skyluc/issue/sbt-2.11.
2015-09-09 09:57:58 +01: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
Timothy Chen b087d23e28 [SPARK-9669] [MESOS] Support PySpark on Mesos cluster mode.
Support running pyspark with cluster mode on Mesos!
This doesn't upload any scripts, so if running in a remote Mesos requires the user to specify the script from a available URI.

Author: Timothy Chen <tnachen@gmail.com>

Closes #8349 from tnachen/mesos_python.
2015-09-04 15:21:31 -07:00
Andrew Or cf42138643 [SPARK-10003] Improve readability of DAGScheduler
Note: this is not intended to be in Spark 1.5!

This patch rewrites some code in the `DAGScheduler` to make it more readable. In particular
- there were blocks of code that are unnecessary and removed for simplicity
- there were abstractions that are unnecessary and made the code hard to navigate
- other minor changes

Author: Andrew Or <andrew@databricks.com>

Closes #8217 from andrewor14/dag-scheduler-readability and squashes the following commits:

57abca3 [Andrew Or] Move comment back into if case
574fb1e [Andrew Or] Merge branch 'master' of github.com:apache/spark into dag-scheduler-readability
64a9ed2 [Andrew Or] Remove unnecessary code + minor code rewrites
2015-09-03 17:55:10 -07:00
Andrew Or 08b0750974 [SPARK-10435] Spark submit should fail fast for Mesos cluster mode with R
It's not supported yet so we should error with a clear message.

Author: Andrew Or <andrew@databricks.com>

Closes #8590 from andrewor14/mesos-cluster-r-guard.
2015-09-03 13:57:20 -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
CHOIJAEHONG af0e3125cb [SPARK-8951] [SPARKR] support Unicode characters in collect()
Spark gives an error message and does not show the output when a field of the result DataFrame contains characters in CJK.
I changed SerDe.scala in order that Spark support Unicode characters when writes a string to R.

Author: CHOIJAEHONG <redrock07@naver.com>

Closes #7494 from CHOIJAEHONG1/SPARK-8951.
2015-09-03 13:38:26 -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
Evan Racah f6c447f875 Removed code duplication in ShuffleBlockFetcherIterator
Added fetchUpToMaxBytes() to prevent having to update both code blocks when a change is made.

Author: Evan Racah <ejracah@gmail.com>

Closes #8514 from eracah/master.
2015-09-02 22:13:37 -07:00
navis.ryu 0985d2c30e [SPARK-8707] RDD#toDebugString fails if any cached RDD has invalid partitions
Added numPartitions(evaluate: Boolean) to RDD. With "evaluate=true" the method is same with "partitions.length". With "evaluate=false", it checks checked-out or already evaluated partitions in the RDD to get number of partition. If it's not those cases, returns -1. RDDInfo.partitionNum calls numPartition only when it's accessed.

Author: navis.ryu <navis@apache.org>

Closes #7127 from navis/SPARK-8707.
2015-09-02 22:11:51 -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
Marcelo Vanzin 2da3a9e98e [SPARK-10004] [SHUFFLE] Perform auth checks when clients read shuffle data.
To correctly isolate applications, when requests to read shuffle data
arrive at the shuffle service, proper authorization checks need to
be performed. This change makes sure that only the application that
created the shuffle data can read from it.

Such checks are only enabled when "spark.authenticate" is enabled,
otherwise there's no secure way to make sure that the client is really
who it says it is.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #8218 from vanzin/SPARK-10004.
2015-09-02 12:53:24 -07:00
Chuan Shao c3b881a7d7 [SPARK-7336] [HISTORYSERVER] Fix bug that applications status incorrect on JobHistory UI.
Author: ArcherShao <shaochuan@huawei.com>

Closes #5886 from ArcherShao/SPARK-7336.
2015-09-02 11:02:57 -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
ihainan 1bfd934782 [SPARK-10184] [CORE] Optimization for bounds determination in RangePartitioner
JIRA Issue: https://issues.apache.org/jira/browse/SPARK-10184

Change `cumWeight > target` to `cumWeight >= target` in `RangePartitioner.determineBounds` method to make the output partitions more balanced.

Author: ihainan <ihainan72@gmail.com>

Closes #8397 from ihainan/opt_for_rangepartitioner.
2015-08-30 08:26:14 +01: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
Sean Owen cc39803062 [SPARK-10295] [CORE] Dynamic allocation in Mesos does not release when RDDs are cached
Remove obsolete warning about dynamic allocation not working with cached RDDs

See discussion in https://issues.apache.org/jira/browse/SPARK-10295

Author: Sean Owen <sowen@cloudera.com>

Closes #8489 from srowen/SPARK-10295.
2015-08-28 09:32:23 +01: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
Sun Rui 71a138cd0e [SPARK-10048] [SPARKR] Support arbitrary nested Java array in serde.
This PR:
1. supports transferring arbitrary nested array from JVM to R side in SerDe;
2. based on 1, collect() implemenation is improved. Now it can support collecting data of complex types
   from a DataFrame.

Author: Sun Rui <rui.sun@intel.com>

Closes #8276 from sun-rui/SPARK-10048.
2015-08-25 13:14:10 -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
ehnalis 7f1e507bf7 Fixed a typo in DAGScheduler.
Author: ehnalis <zoltan.zvara@gmail.com>

Closes #8308 from ehnalis/master.
2015-08-25 12:30:06 +01:00
Zhang, Liye 5c14890159 [DOC] add missing parameters in SparkContext.scala for scala doc
Author: Zhang, Liye <liye.zhang@intel.com>

Closes #8412 from liyezhang556520/minorDoc.
2015-08-25 11:48:55 +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
Xiangrui Meng cdd9a2bb10 [SPARK-10140] [DOC] add target fields to @Since
so constructors parameters and public fields can be annotated. rxin MechCoder

Author: Xiangrui Meng <meng@databricks.com>

Closes #8344 from mengxr/SPARK-10140.2.
2015-08-20 20:01:13 -07:00
Alex Shkurenko 39e91fe2fd [SPARK-9982] [SPARKR] SparkR DataFrame fail to return data of Decimal type
Author: Alex Shkurenko <ashkurenko@enova.com>

Closes #8239 from ashkurenko/master.
2015-08-20 10:16:38 -07:00
Timothy Chen 73431d8afb [SPARK-10124] [MESOS] Fix removing queued driver in mesos cluster mode.
Currently the spark applications can be queued to the Mesos cluster dispatcher, but when multiple jobs are in queue we don't handle removing jobs from the buffer correctly while iterating and causes null pointer exception.

This patch copies the buffer before iterating them, so exceptions aren't thrown when the jobs are removed.

Author: Timothy Chen <tnachen@gmail.com>

Closes #8322 from tnachen/fix_cluster_mode.
2015-08-19 19:43:26 -07: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