Commit graph

1154 commits

Author SHA1 Message Date
Sean Owen bfa614b127 SPARK-4022 [CORE] [MLLIB] Replace colt dependency (LGPL) with commons-math
This change replaces usages of colt with commons-math3 equivalents, and makes some minor necessary adjustments to related code and tests to match.

Author: Sean Owen <sowen@cloudera.com>

Closes #2928 from srowen/SPARK-4022 and squashes the following commits:

61a232f [Sean Owen] Fix failure due to different sampling in JavaAPISuite.sample()
16d66b8 [Sean Owen] Simplify seeding with call to reseedRandomGenerator
a1a78e0 [Sean Owen] Use Well19937c
31c7641 [Sean Owen] Fix Python Poisson test by choosing a different seed; about 88% of seeds should work but 1 didn't, it seems
5c9c67f [Sean Owen] Additional test fixes from review
d8f88e0 [Sean Owen] Replace colt with commons-math3. Some tests do not pass yet.
2014-10-27 10:53:15 -07:00
Sandy Ryza dea302ddbd SPARK-2621. Update task InputMetrics incrementally
The patch takes advantage an API provided in Hadoop 2.5 that allows getting accurate data on Hadoop FileSystem bytes read.  It eliminates the old method, which naively accepts the split size as the input bytes.  An impact of this change will be that input metrics go away when using against Hadoop versions earlier thatn 2.5.  I can add this back in, but my opinion is that no metrics are better than inaccurate metrics.

This is difficult to write a test for because we don't usually build against a version of Hadoop that contains the function we need.  I've tested it manually on a pseudo-distributed cluster.

Author: Sandy Ryza <sandy@cloudera.com>

Closes #2087 from sryza/sandy-spark-2621 and squashes the following commits:

23010b8 [Sandy Ryza] Missing style fixes
74fc9bb [Sandy Ryza] Make getFSBytesReadOnThreadCallback private
1ab662d [Sandy Ryza] Clear things up a bit
984631f [Sandy Ryza] Switch from pull to push model and add test
7ef7b22 [Sandy Ryza] Add missing curly braces
219abc9 [Sandy Ryza] Fall back to split size
90dbc14 [Sandy Ryza] SPARK-2621. Update task InputMetrics incrementally
2014-10-27 10:04:24 -07:00
Shivaram Venkataraman 9aa340a23f [SPARK-4030] Make destroy public for broadcast variables
This change makes the destroy function public for broadcast variables. Motivation for the change is described in https://issues.apache.org/jira/browse/SPARK-4030.
This patch also logs where destroy was called from if a broadcast variable is used after destruction.

Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu>

Closes #2922 from shivaram/broadcast-destroy and squashes the following commits:

a11abab [Shivaram Venkataraman] Fix scala style in Utils.scala
bed9c9d [Shivaram Venkataraman] Make destroy blocking by default
e80c1ab [Shivaram Venkataraman] Make destroy public for broadcast variables Also log where destroy was called from if a broadcast variable is used after destruction.
2014-10-27 08:45:36 -07:00
Josh Rosen bf589fc717 [SPARK-3616] Add basic Selenium tests to WebUISuite
This patch adds Selenium tests for Spark's web UI.  To avoid adding extra
dependencies to the test environment, the tests use Selenium's HtmlUnitDriver,
which is pure-Java, instead of, say, ChromeDriver.

I added new tests to try to reproduce a few UI bugs reported on JIRA, namely
SPARK-3021, SPARK-2105, and SPARK-2527.  I wasn't able to reproduce these bugs;
I suspect that the older ones might have been fixed by other patches.

In order to use HtmlUnitDriver, I added an explicit dependency on the
org.apache.httpcomponents version of httpclient in order to prevent jets3t's
older version from taking precedence on the classpath.

I also upgraded ScalaTest to 2.2.1.

Author: Josh Rosen <joshrosen@apache.org>
Author: Josh Rosen <joshrosen@databricks.com>

Closes #2474 from JoshRosen/webui-selenium-tests and squashes the following commits:

fcc9e83 [Josh Rosen] scalautils -> scalactic package rename
510e54a [Josh Rosen] [SPARK-3616] Add basic Selenium tests to WebUISuite.
2014-10-26 11:29:27 -07:00
Josh Rosen 9530316887 [SPARK-2321] Stable pull-based progress / status API
This pull request is a first step towards the implementation of a stable, pull-based progress / status API for Spark (see [SPARK-2321](https://issues.apache.org/jira/browse/SPARK-2321)).  For now, I'd like to discuss the basic implementation, API names, and overall interface design.  Once we arrive at a good design, I'll go back and add additional methods to expose more information via these API.

#### Design goals:

- Pull-based API
- Usable from Java / Scala / Python (eventually, likely with a wrapper)
- Can be extended to expose more information without introducing binary incompatibilities.
- Returns immutable objects.
- Don't leak any implementation details, preserving our freedom to change the implementation.

#### Implementation:

- Add public methods (`getJobInfo`, `getStageInfo`) to SparkContext to allow status / progress information to be retrieved.
- Add public interfaces (`SparkJobInfo`, `SparkStageInfo`) for our API return values.  These interfaces consist entirely of Java-style getter methods.  The interfaces are currently implemented in Java.  I decided to explicitly separate the interface from its implementation (`SparkJobInfoImpl`, `SparkStageInfoImpl`) in order to prevent users from constructing these responses themselves.
-Allow an existing JobProgressListener to be used when constructing a live SparkUI.  This allows us to re-use this listeners in the implementation of this status API.  There are a few reasons why this listener re-use makes sense:
   - The status API and web UI are guaranteed to show consistent information.
   - These listeners are already well-tested.
   - The same garbage-collection / information retention configurations can apply to both this API and the web UI.
- Extend JobProgressListener to maintain `jobId -> Job` and `stageId -> Stage` mappings.

The progress API methods are implemented in a separate trait that's mixed into SparkContext.  This helps to avoid SparkContext.scala from becoming larger and more difficult to read.

Author: Josh Rosen <joshrosen@databricks.com>
Author: Josh Rosen <joshrosen@apache.org>

Closes #2696 from JoshRosen/progress-reporting-api and squashes the following commits:

e6aa78d [Josh Rosen] Add tests.
b585c16 [Josh Rosen] Accept SparkListenerBus instead of more specific subclasses.
c96402d [Josh Rosen] Address review comments.
2707f98 [Josh Rosen] Expose current stage attempt id
c28ba76 [Josh Rosen] Update demo code:
646ff1d [Josh Rosen] Document spark.ui.retainedJobs.
7f47d6d [Josh Rosen] Clean up SparkUI constructors, per Andrew's feedback.
b77b3d8 [Josh Rosen] Merge remote-tracking branch 'origin/master' into progress-reporting-api
787444c [Josh Rosen] Move status API methods into trait that can be mixed into SparkContext.
f9a9a00 [Josh Rosen] More review comments:
3dc79af [Josh Rosen] Remove creation of unused listeners in SparkContext.
249ca16 [Josh Rosen] Address several review comments:
da5648e [Josh Rosen] Add example of basic progress reporting in Java.
7319ffd [Josh Rosen] Add getJobIdsForGroup() and num*Tasks() methods.
cc568e5 [Josh Rosen] Add note explaining that interfaces should not be implemented outside of Spark.
6e840d4 [Josh Rosen] Remove getter-style names and "consistent snapshot" semantics:
08cbec9 [Josh Rosen] Begin to sketch the interfaces for a stable, public status API.
ac2d13a [Josh Rosen] Add jobId->stage, stageId->stage mappings in JobProgressListener
24de263 [Josh Rosen] Create UI listeners in SparkContext instead of in Tabs:
2014-10-25 00:06:57 -07:00
Kousuke Saruta 098f83c7cc [SPARK-4075] [Deploy] Jar url validation is not enough for Jar file
In deploy.ClientArguments.isValidJarUrl, the url is checked as follows.

    def isValidJarUrl(s: String): Boolean = s.matches("(.+):(.+)jar")

So, it allows like 'hdfs:file.jar' (no authority).

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #2925 from sarutak/uri-syntax-check-improvement and squashes the following commits:

cf06173 [Kousuke Saruta] Improved URI syntax checking
2014-10-24 13:08:59 -07:00
Josh Rosen 83b7a1c650 [SPARK-4019] [SPARK-3740] Fix MapStatus compression bug that could lead to empty results or Snappy errors
This commit fixes a bug in MapStatus that could cause jobs to wrongly return
empty results if those jobs contained stages with more than 2000 partitions
where most of those partitions were empty.

For jobs with > 2000 partitions, MapStatus uses HighlyCompressedMapStatus,
which only stores the average size of blocks.  If the average block size is
zero, then this will cause all blocks to be reported as empty, causing
BlockFetcherIterator to mistakenly skip them.

For example, this would return an empty result:

    sc.makeRDD(0 until 10, 1000).repartition(2001).collect()

This can also lead to deserialization errors (e.g. Snappy decoding errors)
for jobs with > 2000 partitions where the average block size is non-zero but
there is at least one empty block.  In this case, the BlockFetcher attempts to
fetch empty blocks and fails when trying to deserialize them.

The root problem here is that MapStatus has a (previously undocumented)
correctness property that was violated by HighlyCompressedMapStatus:

    If a block is non-empty, then getSizeForBlock must be non-zero.

I fixed this by modifying HighlyCompressedMapStatus to store the average size
of _non-empty_ blocks and to use a compressed bitmap to track which blocks are
empty.

I also removed a test which was broken as originally written: it attempted
to check that HighlyCompressedMapStatus's size estimation error was < 10%,
but this was broken because HighlyCompressedMapStatus is only used for map
statuses with > 2000 partitions, but the test only created 50.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #2866 from JoshRosen/spark-4019 and squashes the following commits:

fc8b490 [Josh Rosen] Roll back hashset change, which didn't improve performance.
5faa0a4 [Josh Rosen] Incorporate review feedback
c8b8cae [Josh Rosen] Two performance fixes:
3b892dd [Josh Rosen] Address Reynold's review comments
ba2e71c [Josh Rosen] Add missing newline
609407d [Josh Rosen] Use Roaring Bitmap to track non-empty blocks.
c23897a [Josh Rosen] Use sets when comparing collect() results
91276a3 [Josh Rosen] [SPARK-4019] Fix MapStatus compression bug that could lead to empty results.
2014-10-23 16:39:32 -07:00
Josh Rosen 813effc701 [SPARK-3426] Fix sort-based shuffle error when spark.shuffle.compress and spark.shuffle.spill.compress settings are different
This PR fixes SPARK-3426, an issue where sort-based shuffle crashes if the
`spark.shuffle.spill.compress` and `spark.shuffle.compress` settings have
different values.

The problem is that sort-based shuffle's read and write paths use different
settings for determining whether to apply compression.  ExternalSorter writes
runs to files using `TempBlockId` ids, which causes
`spark.shuffle.spill.compress` to be used for enabling compression, but these
spilled files end up being shuffled over the network and read as shuffle files
using `ShuffleBlockId` by BlockStoreShuffleFetcher, which causes
`spark.shuffle.compress` to be used for enabling decompression.  As a result,
this leads to errors when these settings disagree.

Based on the discussions in #2247 and #2178, it sounds like we don't want to
remove the `spark.shuffle.spill.compress` setting.  Therefore, I've tried to
come up with a fix where `spark.shuffle.spill.compress` is used to compress
data that's read and written locally and `spark.shuffle.compress` is used to
compress any data that will be fetched / read as shuffle blocks.

To do this, I split `TempBlockId` into two new id types, `TempLocalBlockId` and
`TempShuffleBlockId`, which map to `spark.shuffle.spill.compress` and
`spark.shuffle.compress`, respectively.  ExternalAppendOnlyMap also used temp
blocks for spilling data.  It looks like ExternalSorter was designed to be
a generic sorter but its configuration already happens to be tied to sort-based
shuffle, so I think it's fine if we use `spark.shuffle.compress` to compress
its spills; we can move the compression configuration to the constructor in
a later commit if we find that ExternalSorter is being used in other contexts
where we want different configuration options to control compression.  To
summarize:

**Before:**

|       | ExternalAppendOnlyMap        | ExternalSorter               |
|-------|------------------------------|------------------------------|
| Read  | spark.shuffle.spill.compress | spark.shuffle.compress       |
| Write | spark.shuffle.spill.compress | spark.shuffle.spill.compress |

**After:**

|       | ExternalAppendOnlyMap        | ExternalSorter         |
|-------|------------------------------|------------------------|
| Read  | spark.shuffle.spill.compress | spark.shuffle.compress |
| Write | spark.shuffle.spill.compress | spark.shuffle.compress |

Thanks to andrewor14 for debugging this with me!

Author: Josh Rosen <joshrosen@databricks.com>

Closes #2890 from JoshRosen/SPARK-3426 and squashes the following commits:

1921cf6 [Josh Rosen] Minor edit for clarity.
c8dd8f2 [Josh Rosen] Add comment explaining use of createTempShuffleBlock().
2c687b9 [Josh Rosen] Fix SPARK-3426.
91e7e40 [Josh Rosen] Combine tests into single test of all combinations
76ca65e [Josh Rosen] Add regression test for SPARK-3426.
2014-10-22 14:49:58 -07:00
Sandy Ryza 6bb56faea8 SPARK-1813. Add a utility to SparkConf that makes using Kryo really easy
Author: Sandy Ryza <sandy@cloudera.com>

Closes #789 from sryza/sandy-spark-1813 and squashes the following commits:

48b05e9 [Sandy Ryza] Simplify
b824932 [Sandy Ryza] Allow both spark.kryo.classesToRegister and spark.kryo.registrator at the same time
6a15bb7 [Sandy Ryza] Small fix
a2278c0 [Sandy Ryza] Respond to review comments
6ef592e [Sandy Ryza] SPARK-1813. Add a utility to SparkConf that makes using Kryo really easy
2014-10-21 21:53:09 -07:00
Josh Rosen 5a8f64f336 [SPARK-3958] TorrentBroadcast cleanup / debugging improvements.
This PR makes several changes to TorrentBroadcast in order to make
it easier to reason about, which should help when debugging SPARK-3958.
The key changes:

- Remove all state from the global TorrentBroadcast object.  This state
  consisted mainly of configuration options, like the block size and
  compression codec, and was read by the blockify / unblockify methods.
  Unfortunately, the use of `lazy val` for `BLOCK_SIZE` meant that the block
  size was always determined by the first SparkConf that TorrentBroadast was
  initialized with; as a result, unit tests could not properly test
  TorrentBroadcast with different block sizes.

  Instead, blockifyObject and unBlockifyObject now accept compression codecs
  and blockSizes as arguments.  These arguments are supplied at the call sites
  inside of TorrentBroadcast instances.  Each TorrentBroadcast instance
  determines these values from SparkEnv's SparkConf.  I was careful to ensure
  that we do not accidentally serialize CompressionCodec or SparkConf objects
  as part of the TorrentBroadcast object.

- Remove special-case handling of local-mode in TorrentBroadcast.  I don't
  think that broadcast implementations should know about whether we're running
  in local mode.  If we want to optimize the performance of broadcast in local
  mode, then we should detect this at a higher level and use a dummy
  LocalBroadcastFactory implementation instead.

  Removing this code fixes a subtle error condition: in the old local mode
  code, a failure to find the broadcast in the local BlockManager would lead
  to an attempt to deblockify zero blocks, which could lead to confusing
  deserialization or decompression errors when we attempted to decompress
  an empty byte array.  This should never have happened, though: a failure to
  find the block in local mode is evidence of some other error.  The changes
  here will make it easier to debug those errors if they ever happen.

- Add a check that throws an exception when attempting to deblockify an
  empty array.

- Use ScalaCheck to add a test to check that TorrentBroadcast's
  blockifyObject and unBlockifyObject methods are inverses.

- Misc. cleanup and logging improvements.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #2844 from JoshRosen/torrentbroadcast-bugfix and squashes the following commits:

1e8268d [Josh Rosen] Address Reynold's review comments
2a9fdfd [Josh Rosen] Address Reynold's review comments.
c3b08f9 [Josh Rosen] Update TorrentBroadcast tests to reflect removal of special local-mode optimizations.
5c22782 [Josh Rosen] Store broadcast variable's value in the driver.
33fc754 [Josh Rosen] Change blockify/unblockifyObject to accept serializer as argument.
618a872 [Josh Rosen] [SPARK-3958] TorrentBroadcast cleanup / debugging improvements.
2014-10-21 00:49:11 -07:00
Josh Rosen d1966f3a8b [SPARK-3902] [SPARK-3590] Stabilize AsynRDDActions and add Java API
This PR adds a Java API for AsyncRDDActions and promotes the API from `Experimental` to stable.

Author: Josh Rosen <joshrosen@apache.org>
Author: Josh Rosen <joshrosen@databricks.com>

Closes #2760 from JoshRosen/async-rdd-actions-in-java and squashes the following commits:

0d45fbc [Josh Rosen] Whitespace fix.
ad3ae53 [Josh Rosen] Merge remote-tracking branch 'origin/master' into async-rdd-actions-in-java
c0153a5 [Josh Rosen] Remove unused variable.
e8e2867 [Josh Rosen] Updates based on Marcelo's review feedback
7a1417f [Josh Rosen] Removed unnecessary java.util import.
6f8f6ac [Josh Rosen] Fix import ordering.
ff28e49 [Josh Rosen] Add MiMa excludes and fix a scalastyle error.
346e46e [Josh Rosen] [SPARK-3902] Stabilize AsyncRDDActions; add Java API.
2014-10-19 20:02:31 -07:00
Prashant Sharma 2fe0ba9561 SPARK-3874: Provide stable TaskContext API
This is a small number of clean-up changes on top of #2782. Closes #2782.

Author: Prashant Sharma <prashant.s@imaginea.com>
Author: Patrick Wendell <pwendell@gmail.com>

Closes #2803 from pwendell/pr-2782 and squashes the following commits:

56d5b7a [Patrick Wendell] Minor clean-up
44089ec [Patrick Wendell] Clean-up the TaskContext API.
ed551ce [Prashant Sharma] Fixed a typo
df261d0 [Prashant Sharma] Josh's suggestion
facf3b1 [Prashant Sharma] Fixed the mima issue.
7ecc2fe [Prashant Sharma] CR, Moved implementations to TaskContextImpl
bbd9e05 [Prashant Sharma] adding missed out files to git.
ef633f5 [Prashant Sharma] SPARK-3874, Provide stable TaskContext API
2014-10-16 21:38:45 -04:00
GuoQiang Li 293a0b5dbb [SPARK-2098] All Spark processes should support spark-defaults.conf, config file
This is another implementation about #1256
cc andrewor14 vanzin

Author: GuoQiang Li <witgo@qq.com>

Closes #2379 from witgo/SPARK-2098-new and squashes the following commits:

4ef1cbd [GuoQiang Li] review commit
49ef70e [GuoQiang Li] Refactor getDefaultPropertiesFile
c45d20c [GuoQiang Li] All Spark processes should support spark-defaults.conf, config file
2014-10-14 22:16:38 -07:00
Bill Bejeck 9b6de6fbc0 SPARK-3178 setting SPARK_WORKER_MEMORY to a value without a label (m or g) sets the worker memory limit to zero
Validate the memory is greater than zero when set from the SPARK_WORKER_MEMORY environment variable or command line without a g or m label.  Added unit tests. If memory is 0 an IllegalStateException is thrown. Updated unit tests to mock environment variables by subclassing SparkConf (tip provided by Josh Rosen).   Updated WorkerArguments to use SparkConf.getenv instead of System.getenv for reading the SPARK_WORKER_MEMORY environment variable.

Author: Bill Bejeck <bbejeck@gmail.com>

Closes #2309 from bbejeck/spark-memory-worker and squashes the following commits:

51cf915 [Bill Bejeck] SPARK-3178 - Validate the memory is greater than zero when set from the SPARK_WORKER_MEMORY environment variable or command line without a g or m label.  Added unit tests. If memory is 0 an IllegalStateException is thrown. Updated unit tests to mock environment variables by subclassing SparkConf (tip provided by Josh Rosen).   Updated WorkerArguments to use SparkConf.getenv instead of System.getenv for reading the SPARK_WORKER_MEMORY environment variable.
2014-10-14 12:12:38 -07:00
Aaron Davidson 186b497c94 [SPARK-3921] Fix CoarseGrainedExecutorBackend's arguments for Standalone mode
The goal of this patch is to fix the swapped arguments in standalone mode, which was caused by  79e45c9323 (diff-79391110e9f26657e415aa169a004998R153).

More details can be found in the JIRA: [SPARK-3921](https://issues.apache.org/jira/browse/SPARK-3921)

Tested in Standalone mode, but not in Mesos.

Author: Aaron Davidson <aaron@databricks.com>

Closes #2779 from aarondav/fix-standalone and squashes the following commits:

725227a [Aaron Davidson] Fix ExecutorRunnerTest
9d703fe [Aaron Davidson] [SPARK-3921] Fix CoarseGrainedExecutorBackend's arguments for Standalone mode
2014-10-13 23:31:37 -07:00
Jakub Dubovský fc616d51a5 [SPARK-3121] Wrong implementation of implicit bytesWritableConverter
val path = ... //path to seq file with BytesWritable as type of both key and value
val file = sc.sequenceFile[Array[Byte],Array[Byte]](path)
file.take(1)(0)._1

This prints incorrect content of byte array. Actual content starts with correct one and some "random" bytes and zeros are appended. BytesWritable has two methods:

getBytes() - return content of all internal array which is often longer then actual value stored. It usually contains the rest of previous longer values

copyBytes() - return just begining of internal array determined by internal length property

It looks like in implicit conversion between BytesWritable and Array[byte] getBytes is used instead of correct copyBytes.

dbtsai

Author: Jakub Dubovský <james64@inMail.sk>
Author: Dubovsky Jakub <dubovsky@avast.com>

Closes #2712 from james64/3121-bugfix and squashes the following commits:

f85d24c [Jakub Dubovský] Test name changed, comments added
1b20d51 [Jakub Dubovský] Import placed correctly
406e26c [Jakub Dubovský] Scala style fixed
f92ffa6 [Dubovsky Jakub] performance tuning
480f9cd [Dubovsky Jakub] Bug 3121 fixed
2014-10-12 22:03:26 -07:00
Josh Rosen 18bd67c24b [SPARK-3887] Send stracktrace in ConnectionManager error replies
When reporting that a remote error occurred, the ConnectionManager should also log the stacktrace of the remote exception. This PR accomplishes this by sending the remote exception's stacktrace as the payload in the "negative ACK / error message."

Author: Josh Rosen <joshrosen@apache.org>

Closes #2741 from JoshRosen/propagate-cm-exceptions-to-sender and squashes the following commits:

b5366cc [Josh Rosen] Explicitly encode error messages using UTF-8.
cef18b3 [Josh Rosen] [SPARK-3887] Send stracktrace in ConnectionManager error messages.
2014-10-12 13:08:42 -07:00
Prashant Sharma 0e8203f4fb [SPARK-2924] Required by scala 2.11, only one fun/ctor amongst overriden alternatives, can have default argument(s).
...riden alternatives, can have default argument.

Author: Prashant Sharma <prashant.s@imaginea.com>

Closes #2750 from ScrapCodes/SPARK-2924/default-args-removed and squashes the following commits:

d9785c3 [Prashant Sharma] [SPARK-2924] Required by scala 2.11, only one function/ctor amongst overriden alternatives, can have default argument.
2014-10-10 18:39:55 -07:00
Anand Avati 411cf29fff [SPARK-2805] Upgrade Akka to 2.3.4
This is a second rev of the Akka upgrade (earlier merged, but reverted). I made a slight modification which is that I also upgrade Hive to deal with a compatibility issue related to the protocol buffers library.

Author: Anand Avati <avati@redhat.com>
Author: Patrick Wendell <pwendell@gmail.com>

Closes #2752 from pwendell/akka-upgrade and squashes the following commits:

4c7ca3f [Patrick Wendell] Upgrading to new hive->protobuf version
57a2315 [Anand Avati] SPARK-1812: streaming - remove tests which depend on akka.actor.IO
2a551d3 [Anand Avati] SPARK-1812: core - upgrade to akka 2.3.4
2014-10-10 00:46:56 -07:00
Sean Owen 363baacade SPARK-3811 [CORE] More robust / standard Utils.deleteRecursively, Utils.createTempDir
I noticed a few issues with how temp directories are created and deleted:

*Minor*

* Guava's `Files.createTempDir()` plus `File.deleteOnExit()` is used in many tests to make a temp dir, but `Utils.createTempDir()` seems to be the standard Spark mechanism
* Call to `File.deleteOnExit()` could be pushed into `Utils.createTempDir()` as well, along with this replacement
* _I messed up the message in an exception in `Utils` in SPARK-3794; fixed here_

*Bit Less Minor*

* `Utils.deleteRecursively()` fails immediately if any `IOException` occurs, instead of trying to delete any remaining files and subdirectories. I've observed this leave temp dirs around. I suggest changing it to continue in the face of an exception and throw one of the possibly several exceptions that occur at the end.
* `Utils.createTempDir()` will add a JVM shutdown hook every time the method is called. Even if the subdir is the parent of another parent dir, since this check is inside the hook. However `Utils` manages a set of all dirs to delete on shutdown already, called `shutdownDeletePaths`. A single hook can be registered to delete all of these on exit. This is how Tachyon temp paths are cleaned up in `TachyonBlockManager`.

I noticed a few other things that might be changed but wanted to ask first:

* Shouldn't the set of dirs to delete be `File`, not just `String` paths?
* `Utils` manages the set of `TachyonFile` that have been registered for deletion, but the shutdown hook is managed in `TachyonBlockManager`. Should this logic not live together, and not in `Utils`? it's more specific to Tachyon, and looks a slight bit odd to import in such a generic place.

Author: Sean Owen <sowen@cloudera.com>

Closes #2670 from srowen/SPARK-3811 and squashes the following commits:

071ae60 [Sean Owen] Update per @vanzin's review
da0146d [Sean Owen] Make Utils.deleteRecursively try to delete all paths even when an exception occurs; use one shutdown hook instead of one per method call to delete temp dirs
3a0faa4 [Sean Owen] Standardize on Utils.createTempDir instead of Files.createTempDir
2014-10-09 18:21:59 -07:00
Patrick Wendell 1faa1135a3 Revert "[SPARK-2805] Upgrade to akka 2.3.4"
This reverts commit b9df8af62e.
2014-10-09 14:50:36 -07:00
Anand Avati b9df8af62e [SPARK-2805] Upgrade to akka 2.3.4
Upgrade to akka 2.3.4

Author: Anand Avati <avati@redhat.com>

Closes #1685 from avati/SPARK-1812-akka-2.3 and squashes the following commits:

57a2315 [Anand Avati] SPARK-1812: streaming - remove tests which depend on akka.actor.IO
2a551d3 [Anand Avati] SPARK-1812: core - upgrade to akka 2.3.4
2014-10-08 23:45:21 -07:00
mcheah cf1d32e3e1 [SPARK-1860] More conservative app directory cleanup.
First contribution to the project, so apologize for any significant errors.

This PR addresses [SPARK-1860]. The application directories are now cleaned up in a more conservative manner.

Previously, app-* directories were cleaned up if the directory's timestamp was older than a given time. However, the timestamp on a directory does not reflect the modification times of the files in that directory. Therefore, app-* directories were wiped out even if the files inside them were created recently and possibly being used by Executor tasks.

The solution is to change the cleanup logic to inspect all files within the app-* directory and only eliminate the app-* directory if all files in the directory are stale.

Author: mcheah <mcheah@palantir.com>

Closes #2609 from mccheah/worker-better-app-dir-cleanup and squashes the following commits:

87b5d03 [mcheah] [SPARK-1860] Using more string interpolation. Better error logging.
802473e [mcheah] [SPARK-1860] Cleaning up the logs generated when cleaning directories.
e0a1f2e [mcheah] [SPARK-1860] Fixing broken unit test.
77a9de0 [mcheah] [SPARK-1860] More conservative app directory cleanup.
2014-10-03 14:22:52 -07:00
Kousuke Saruta 79e45c9323 [SPARK-3377] [SPARK-3610] Metrics can be accidentally aggregated / History server log name should not be based on user input
This PR is another solution for #2250

I'm using codahale base MetricsSystem of Spark with JMX or Graphite, and I saw following 2 problems.

(1) When applications which have same spark.app.name run on cluster at the same time, some metrics names are mixed. For instance, if 2+ application is running on the cluster at the same time, each application emits the same named metric like "SparkPi.DAGScheduler.stage.failedStages" and Graphite cannot distinguish the metrics is for which application.

(2) When 2+ executors run on the same machine, JVM metrics of each executors are mixed. For instance, 2+ executors running on the same node can emit the same named metric "jvm.memory" and Graphite cannot distinguish the metrics is from which application.

And there is an similar issue. The directory for event logs is named using application name.
Application name is defined by user and the name can includes illegal character for path names.
Further more, the directory name consists of application name and System.currentTimeMillis even though each application has unique Application ID so if we run jobs which have same name, it's difficult to identify which directory is for which application.

Closes #2250
Closes #1067

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #2432 from sarutak/metrics-structure-improvement2 and squashes the following commits:

3288b2b [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
39169e4 [Kousuke Saruta] Fixed style
6570494 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
817e4f0 [Kousuke Saruta] Simplified MetricsSystem#buildRegistryName
67fa5eb [Kousuke Saruta] Unified MetricsSystem#registerSources and registerSinks in start
10be654 [Kousuke Saruta] Fixed style.
990c078 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
f0c7fba [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
59cc2cd [Kousuke Saruta] Modified SparkContextSchedulerCreationSuite
f9b6fb3 [Kousuke Saruta] Modified style.
2cf8a0f [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
389090d [Kousuke Saruta] Replaced taskScheduler.applicationId() with getApplicationId in SparkContext#postApplicationStart
ff45c89 [Kousuke Saruta] Added some test cases to MetricsSystemSuite
69c46a6 [Kousuke Saruta] Added warning logging logic to MetricsSystem#buildRegistryName
5cca0d2 [Kousuke Saruta] Added Javadoc comment to SparkContext#getApplicationId
16a9f01 [Kousuke Saruta] Added data types to be returned to some methods
6434b06 [Kousuke Saruta] Reverted changes related to ApplicationId
0413b90 [Kousuke Saruta] Deleted ApplicationId.java and ApplicationIdSuite.java
a42300c [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
0fc1b09 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
42bea55 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
248935d [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
f6af132 [Kousuke Saruta] Modified SchedulerBackend and TaskScheduler to return System.currentTimeMillis as an unique Application Id
1b8b53e [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
97cb85c [Kousuke Saruta] Modified confliction of MimExcludes
2cdd009 [Kousuke Saruta] Modified defailt implementation of applicationId
9aadb0b [Kousuke Saruta] Modified NetworkReceiverSuite to ensure "executor.start()" is finished in test "network receiver life cycle"
3011efc [Kousuke Saruta] Added ApplicationIdSuite.scala
d009c55 [Kousuke Saruta] Modified ApplicationId#equals to compare appIds
dfc83fd [Kousuke Saruta] Modified ApplicationId to implement Serializable
9ff4851 [Kousuke Saruta] Modified MimaExcludes.scala to ignore createTaskScheduler method in SparkContext
4567ffc [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
6a91b14 [Kousuke Saruta] Modified SparkContextSchedulerCreationSuite, ExecutorRunnerTest and EventLoggingListenerSuite
0325caf [Kousuke Saruta] Added ApplicationId.scala
0a2fc14 [Kousuke Saruta] Modified style
eabda80 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
0f890e6 [Kousuke Saruta] Modified SparkDeploySchedulerBackend and Master to pass baseLogDir instead f eventLogDir
bcf25bf [Kousuke Saruta] Modified directory name for EventLogs
28d4d93 [Kousuke Saruta] Modified SparkContext and EventLoggingListener so that the directory for EventLogs is named same for Application ID
203634e [Kousuke Saruta] Modified comment in SchedulerBackend#applicationId and TaskScheduler#applicationId
424fea4 [Kousuke Saruta] Modified  the subclasses of TaskScheduler and SchedulerBackend so that they can return non-optional Unique Application ID
b311806 [Kousuke Saruta] Swapped last 2 arguments passed to CoarseGrainedExecutorBackend
8a2b6ec [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
086ee25 [Kousuke Saruta] Merge branch 'metrics-structure-improvement2' of github.com:sarutak/spark into metrics-structure-improvement2
e705386 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
36d2f7a [Kousuke Saruta] Added warning message for the situation we cannot get application id for the prefix for the name of metrics
eea6e19 [Kousuke Saruta] Modified CoarseGrainedMesosSchedulerBackend and MesosSchedulerBackend so that we can get Application ID
c229fbe [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
e719c39 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
4a93c7f [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
4776f9e [Kousuke Saruta] Modified MetricsSystemSuite.scala
efcb6e1 [Kousuke Saruta] Modified to add application id to metrics name
2ec848a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement
3ea7896 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement
ead8966 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement
08e627e [Kousuke Saruta] Revert "tmp"
7b67f5a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement
45bd33d [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement
93e263a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement
848819c [Kousuke Saruta] Merge branch 'metrics-structure-improvement' of github.com:sarutak/spark into metrics-structure-improvement
912a637 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement
e4a4593 [Kousuke Saruta] tmp
3e098d8 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement
4603a39 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement
fa7175b [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement
15f88a3 [Kousuke Saruta] Modified MetricsSystem#buildRegistryName because conf.get does not return null when correspondin entry is absent
6f7dcd4 [Kousuke Saruta] Modified constructor of DAGSchedulerSource and BlockManagerSource because the instance of SparkContext is no longer used
6fc5560 [Kousuke Saruta] Modified sourceName of ExecutorSource, DAGSchedulerSource and BlockManagerSource
4e057c9 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement
85ffc02 [Kousuke Saruta] Revert "Modified sourceName of ExecutorSource, DAGSchedulerSource and BlockManagerSource"
868e326 [Kousuke Saruta] Modified MetricsSystem to set registry name with unique application-id and driver/executor-id
71609f5 [Kousuke Saruta] Modified sourceName of ExecutorSource, DAGSchedulerSource and BlockManagerSource
55debab [Kousuke Saruta] Modified SparkContext and Executor to set spark.executor.id to identifiers
4180993 [Kousuke Saruta] Modified SparkContext to retain spark.unique.app.name property in SparkConf
2014-10-03 13:48:56 -07:00
EugenCepoi f0811f928e SPARK-2058: Overriding SPARK_HOME/conf with SPARK_CONF_DIR
Update of PR #997.

With this PR, setting SPARK_CONF_DIR overrides SPARK_HOME/conf (not only spark-defaults.conf and spark-env).

Author: EugenCepoi <cepoi.eugen@gmail.com>

Closes #2481 from EugenCepoi/SPARK-2058 and squashes the following commits:

0bb32c2 [EugenCepoi] use orElse orNull and fixing trailing percent in compute-classpath.cmd
77f35d7 [EugenCepoi] SPARK-2058: Overriding SPARK_HOME/conf with SPARK_CONF_DIR
2014-10-03 10:03:15 -07:00
Tathagata Das 5db78e6b87 [SPARK-3495] Block replication fails continuously when the replication target node is dead AND [SPARK-3496] Block replication by mistake chooses driver as target
If a block manager (say, A) wants to replicate a block and the node chosen for replication (say, B) is dead, then the attempt to send the block to B fails. However, this continues to fail indefinitely. Even if the driver learns about the demise of the B, A continues to try replicating to B and failing miserably.

The reason behind this bug is that A initially fetches a list of peers from the driver (when B was active), but never updates it after B is dead. This affects Spark Streaming as its receiver uses block replication.

The solution in this patch adds the following.
- Changed BlockManagerMaster to return all the peers of a block manager, rather than the requested number. It also filters out driver BlockManager.
- Refactored BlockManager's replication code to handle peer caching correctly.
    + The peer for replication is randomly selected. This is different from past behavior where for a node A, a node B was deterministically chosen for the lifetime of the application.
    + If replication fails to one node, the peers are refetched.
    + The peer cached has a TTL of 1 second to enable discovery of new peers and using them for replication.
- Refactored use of \<driver\> in BlockManager into a new method `BlockManagerId.isDriver`
- Added replication unit tests (replication was not tested till now, duh!)

This should not make a difference in performance of Spark workloads where replication is not used.

@andrewor14 @JoshRosen

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

Closes #2366 from tdas/replication-fix and squashes the following commits:

9690f57 [Tathagata Das] Moved replication tests to a new BlockManagerReplicationSuite.
0661773 [Tathagata Das] Minor changes based on PR comments.
a55a65c [Tathagata Das] Added a unit test to test replication behavior.
012afa3 [Tathagata Das] Bug fix
89f91a0 [Tathagata Das] Minor change.
68e2c72 [Tathagata Das] Made replication peer selection logic more efficient.
08afaa9 [Tathagata Das] Made peer selection for replication deterministic to block id
3821ab9 [Tathagata Das] Fixes based on PR comments.
08e5646 [Tathagata Das] More minor changes.
d402506 [Tathagata Das] Fixed imports.
4a20531 [Tathagata Das] Filtered driver block manager from peer list, and also consolidated the use of <driver> in BlockManager.
7598f91 [Tathagata Das] Minor changes.
03de02d [Tathagata Das] Change replication logic to correctly refetch peers from master on failure and on new worker addition.
d081bf6 [Tathagata Das] Fixed bug in get peers and unit tests to test get-peers and replication under executor churn.
9f0ac9f [Tathagata Das] Modified replication tests to fail on replication bug.
af0c1da [Tathagata Das] Added replication unit tests to BlockManagerSuite
2014-10-02 13:49:47 -07:00
Colin Patrick Mccabe 6e27cb630d SPARK-1767: Prefer HDFS-cached replicas when scheduling data-local tasks
This change reorders the replicas returned by
HadoopRDD#getPreferredLocations so that replicas cached by HDFS are at
the start of the list.  This requires Hadoop 2.5 or higher; previous
versions of Hadoop do not expose the information needed to determine
whether a replica is cached.

Author: Colin Patrick Mccabe <cmccabe@cloudera.com>

Closes #1486 from cmccabe/SPARK-1767 and squashes the following commits:

338d4f8 [Colin Patrick Mccabe] SPARK-1767: Prefer HDFS-cached replicas when scheduling data-local tasks
2014-10-02 00:29:31 -07:00
Marcelo Vanzin 29c3513203 [SPARK-3446] Expose underlying job ids in FutureAction.
FutureAction is the only type exposed through the async APIs, so
for job IDs to be useful they need to be exposed there. The complication
is that some async jobs run more than one job (e.g. takeAsync),
so the exposed ID has to actually be a list of IDs that can actually
change over time. So the interface doesn't look very nice, but...

Change is actually small, I just added a basic test to make sure
it works.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #2337 from vanzin/SPARK-3446 and squashes the following commits:

e166a68 [Marcelo Vanzin] Fix comment.
1fed2bc [Marcelo Vanzin] [SPARK-3446] Expose underlying job ids in FutureAction.
2014-10-01 19:24:22 -07:00
Reynold Xin 3888ee2f38 [SPARK-3748] Log thread name in unit test logs
Thread names are useful for correlating failures.

Author: Reynold Xin <rxin@apache.org>

Closes #2600 from rxin/log4j and squashes the following commits:

83ffe88 [Reynold Xin] [SPARK-3748] Log thread name in unit test logs
2014-10-01 01:03:49 -07:00
Reynold Xin 6b79bfb425 [SPARK-3613] Record only average block size in MapStatus for large stages
This changes the way we send MapStatus from executors back to driver for large stages (>2000 tasks). For large stages, we no longer send one byte per block. Instead, we just send the average block size.

This makes large jobs (tens of thousands of tasks) much more reliable since the driver no longer sends huge amount of data.

Author: Reynold Xin <rxin@apache.org>

Closes #2470 from rxin/mapstatus and squashes the following commits:

822ff54 [Reynold Xin] Code review feedback.
3b86f56 [Reynold Xin] Added MimaExclude.
f89d182 [Reynold Xin] Fixed a bug in MapStatus
6a0401c [Reynold Xin] [SPARK-3613] Record only average block size in MapStatus for large stages.
2014-09-29 22:56:22 -07:00
jerryshao dab1b0ae29 [SPARK-3032][Shuffle] Fix key comparison integer overflow introduced sorting exception
Previous key comparison in `ExternalSorter` will get wrong sorting result or exception when key comparison overflows, details can be seen in [SPARK-3032](https://issues.apache.org/jira/browse/SPARK-3032). Here fix this and add a unit test to prove it.

Author: jerryshao <saisai.shao@intel.com>

Closes #2514 from jerryshao/SPARK-3032 and squashes the following commits:

6f3c302 [jerryshao] Improve the unit test according to comments
01911e6 [jerryshao] Change the test to show the contract violate exception
83acb38 [jerryshao] Minor changes according to comments
fa2a08f [jerryshao] Fix key comparison integer overflow introduced sorting exception
2014-09-29 11:25:32 -07:00
Reynold Xin f350cd3070 [SPARK-3543] TaskContext remaining cleanup work.
Author: Reynold Xin <rxin@apache.org>

Closes #2560 from rxin/TaskContext and squashes the following commits:

9eff95a [Reynold Xin] [SPARK-3543] remaining cleanup work.
2014-09-28 20:32:54 -07:00
Reynold Xin 436a7730b6 Minor cleanup to tighten visibility and remove compilation warning.
Author: Reynold Xin <rxin@apache.org>

Closes #2555 from rxin/cleanup and squashes the following commits:

6add199 [Reynold Xin] Minor cleanup to tighten visibility and remove compilation warning.
2014-09-27 00:57:26 -07:00
Prashant Sharma 5e34855cf0 [SPARK-3543] Write TaskContext in Java and expose it through a static accessor.
Author: Prashant Sharma <prashant.s@imaginea.com>
Author: Shashank Sharma <shashank21j@gmail.com>

Closes #2425 from ScrapCodes/SPARK-3543/withTaskContext and squashes the following commits:

8ae414c [Shashank Sharma] CR
ee8bd00 [Prashant Sharma] Added internal API in docs comments.
ddb8cbe [Prashant Sharma] Moved setting the thread local to where TaskContext is instantiated.
a7d5e23 [Prashant Sharma] Added doc comments.
edf945e [Prashant Sharma] Code review git add -A
f716fd1 [Prashant Sharma] introduced thread local for getting the task context.
333c7d6 [Prashant Sharma] Translated Task context from scala to java.
2014-09-26 21:29:54 -07:00
Aaron Staple 8ca4ecb6a5 [SPARK-546] Add full outer join to RDD and DStream.
leftOuterJoin and rightOuterJoin are already implemented.  This patch adds fullOuterJoin.

Author: Aaron Staple <aaron.staple@gmail.com>

Closes #1395 from staple/SPARK-546 and squashes the following commits:

1f5595c [Aaron Staple] Fix python style
7ac0aa9 [Aaron Staple] [SPARK-546] Add full outer join to RDD and DStream.
3b5d137 [Aaron Staple] In JavaPairDStream, make class tag specification in rightOuterJoin consistent with other functions.
31f2956 [Aaron Staple] Fix left outer join documentation comments.
2014-09-24 20:39:09 -07:00
Marcelo Vanzin 8dfe79ffb2 [SPARK-3647] Add more exceptions to Guava relocation.
Guava's Optional refers to some package private classes / methods, and
when those are relocated the code stops working, throwing exceptions.
So add the affected classes to the exception list too, and add a unit
test.

(Note that this unit test only really makes sense in maven, since we
don't relocate in the sbt build. Also, JavaAPISuite doesn't seem to
be run by "mvn test" - I had to manually add command line options to
enable it.)

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #2496 from vanzin/SPARK-3647 and squashes the following commits:

84f58d7 [Marcelo Vanzin] [SPARK-3647] Add more exceptions to Guava relocation.
2014-09-23 13:42:00 -07:00
Ian Hummel a0454efe21 [SPARK-3595] Respect configured OutputCommitters when calling saveAsHadoopFile
Addresses the issue in https://issues.apache.org/jira/browse/SPARK-3595, namely saveAsHadoopFile hardcoding the OutputCommitter.  This is not ideal when running Spark jobs that write to S3, especially when running them from an EMR cluster where the default OutputCommitter is a DirectOutputCommitter.

Author: Ian Hummel <ian@themodernlife.net>

Closes #2450 from themodernlife/spark-3595 and squashes the following commits:

f37a0e5 [Ian Hummel] Update based on comments from pwendell
a11d9f3 [Ian Hummel] Fix formatting
4359664 [Ian Hummel] Add an example showing usage
8b6be94 [Ian Hummel] Add ability to specify OutputCommitter, espcially useful when writing to an S3 bucket from an EMR cluster
2014-09-21 13:04:36 -07:00
Sandy Ryza 7c8ad1c083 SPARK-3574. Shuffle finish time always reported as -1
The included test waits 100 ms after job completion for task completion events to come in so it can verify they have reasonable finish times.  Does anyone know a better way to wait on listener events that are expected to come in?

Author: Sandy Ryza <sandy@cloudera.com>

Closes #2440 from sryza/sandy-spark-3574 and squashes the following commits:

c81439b [Sandy Ryza] Fix test failure
b340956 [Sandy Ryza] SPARK-3574. Remove shuffleFinishTime metric
2014-09-20 16:03:17 -07:00
Victsm b3ed37e5ba [SPARK-3560] Fixed setting spark.jars system property in yarn-cluster mode
Author: Victsm <victor.nju@gmail.com>
Author: Min Shen <mshen@linkedin.com>

Closes #2449 from Victsm/SPARK-3560 and squashes the following commits:

918405a [Victsm] Removed the additional space
4502a2a [Min Shen] [SPARK-3560] Fixed setting spark.jars system property in yarn-cluster mode.

(cherry picked from commit 832dff64dd)
Signed-off-by: Andrew Or <andrewor14@gmail.com>
2014-09-18 15:58:29 -07:00
WangTaoTheTonic 3f169bfe3c [SPARK-3565]Fix configuration item not consistent with document
https://issues.apache.org/jira/browse/SPARK-3565

"spark.ports.maxRetries" should be "spark.port.maxRetries". Make the configuration keys in document and code consistent.

Author: WangTaoTheTonic <barneystinson@aliyun.com>

Closes #2427 from WangTaoTheTonic/fixPortRetries and squashes the following commits:

c178813 [WangTaoTheTonic] Use blank lines trigger Jenkins
646f3fe [WangTaoTheTonic] also in SparkBuild.scala
3700dba [WangTaoTheTonic] Fix configuration item not consistent with document
2014-09-17 21:59:23 -07:00
Andrew Or 0a7091e689 [SPARK-3555] Fix UISuite race condition
The test "jetty selects different port under contention" is flaky.

If another process binds to 4040 before the test starts, then the first server we start there will fail, and the subsequent servers we start thereafter may successfully bind to 4040 if it was released between the servers starting. Instead, we should just let Java find a random free port for us and hold onto it for the duration of the test.

Author: Andrew Or <andrewor14@gmail.com>

Closes #2418 from andrewor14/fix-port-contention and squashes the following commits:

0cd4974 [Andrew Or] Stop them servers
a7071fe [Andrew Or] Pick random port instead of 4040
2014-09-16 16:03:20 -07:00
Reynold Xin 2584ea5b23 [SPARK-3469] Make sure all TaskCompletionListener are called even with failures
This is necessary because we rely on this callback interface to clean resources up. The old behavior would lead to resource leaks.

Note that this also changes the fault semantics of TaskCompletionListener. Previously failures in TaskCompletionListeners would result in the task being reported immediately. With this change, we report the exception at the end, and the reported exception is a TaskCompletionListenerException that contains all the exception messages.

Author: Reynold Xin <rxin@apache.org>

Closes #2343 from rxin/taskcontext-callback and squashes the following commits:

a3845b2 [Reynold Xin] Mark TaskCompletionListenerException as private[spark].
ac5baea [Reynold Xin] Removed obsolete comment.
aa68ea4 [Reynold Xin] Throw an exception if task completion callback fails.
29b6162 [Reynold Xin] oops compilation failed.
1cb444d [Reynold Xin] [SPARK-3469] Call all TaskCompletionListeners even if some fail.
2014-09-12 21:55:39 -07:00
witgo 33c7a738ae SPARK-2482: Resolve sbt warnings during build
At the same time, import the `scala.language.postfixOps` and ` org.scalatest.time.SpanSugar._` cause `scala.language.postfixOps` doesn't work

Author: witgo <witgo@qq.com>

Closes #1330 from witgo/sbt_warnings3 and squashes the following commits:

179ba61 [witgo] Resolve sbt warnings during build
2014-09-11 18:44:35 -07:00
Andrew Or 6324eb7b5b [Spark-3490] Disable SparkUI for tests
We currently open many ephemeral ports during the tests, and as a result we occasionally can't bind to new ones. This has caused the `DriverSuite` and the `SparkSubmitSuite` to fail intermittently.

By disabling the `SparkUI` when it's not needed, we already cut down on the number of ports opened significantly, on the order of the number of `SparkContexts` ever created. We must keep it enabled for a few tests for the UI itself, however.

Author: Andrew Or <andrewor14@gmail.com>

Closes #2363 from andrewor14/disable-ui-for-tests and squashes the following commits:

332a7d5 [Andrew Or] No need to set spark.ui.port to 0 anymore
30c93a2 [Andrew Or] Simplify streaming UISuite
a431b84 [Andrew Or] Fix streaming test failures
8f5ae53 [Andrew Or] Fix no new line at the end
29c9b5b [Andrew Or] Disable SparkUI for tests
2014-09-11 17:18:46 -07:00
scwf 26862337c9 [SPARK-3193]output errer info when Process exit code is not zero in test suite
https://issues.apache.org/jira/browse/SPARK-3193
I noticed that sometimes pr tests failed due to the Process exitcode != 0,refer to
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/18688/consoleFull
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/19118/consoleFull

[info] SparkSubmitSuite:
[info] - prints usage on empty input
[info] - prints usage with only --help
[info] - prints error with unrecognized options
[info] - handle binary specified but not class
[info] - handles arguments with --key=val
[info] - handles arguments to user program
[info] - handles arguments to user program with name collision
[info] - handles YARN cluster mode
[info] - handles YARN client mode
[info] - handles standalone cluster mode
[info] - handles standalone client mode
[info] - handles mesos client mode
[info] - handles confs with flag equivalents
[info] - launch simple application with spark-submit *** FAILED ***
[info]   org.apache.spark.SparkException: Process List(./bin/spark-submit, --class, org.apache.spark.deploy.SimpleApplicationTest, --name, testApp, --master, local, file:/tmp/1408854098404-0/testJar-1408854098404.jar) exited with code 1
[info]   at org.apache.spark.util.Utils$.executeAndGetOutput(Utils.scala:872)
[info]   at org.apache.spark.deploy.SparkSubmitSuite.runSparkSubmit(SparkSubmitSuite.scala:311)
[info]   at org.apache.spark.deploy.SparkSubmitSuite$$anonfun$14.apply$mcV$sp(SparkSubmitSuite.scala:291)
[info]   at org.apache.spark.deploy.SparkSubmitSuite$$anonfun$14.apply(SparkSubmitSuite.scala:284)
[info]   at org.apacSpark assembly has been built with Hive, including Datanucleus jars on classpath

this PR output the process error info when failed, it can be helpful for diagnosis.

Author: scwf <wangfei1@huawei.com>

Closes #2108 from scwf/output-test-error-info and squashes the following commits:

0c48082 [scwf] minor fix according to comments
563fde1 [scwf] output errer info when Process exitcode not zero
2014-09-09 11:57:01 -07:00
Sandy Ryza 88547a09fc SPARK-3422. JavaAPISuite.getHadoopInputSplits isn't used anywhere.
Author: Sandy Ryza <sandy@cloudera.com>

Closes #2324 from sryza/sandy-spark-3422 and squashes the following commits:

6446175 [Sandy Ryza] SPARK-3422. JavaAPISuite.getHadoopInputSplits isn't used anywhere.
2014-09-09 10:23:28 -07:00
Reynold Xin 08ce18881e [SPARK-3019] Pluggable block transfer interface (BlockTransferService)
This pull request creates a new BlockTransferService interface for block fetch/upload and refactors the existing ConnectionManager to implement BlockTransferService (NioBlockTransferService).

Most of the changes are simply moving code around. The main class to inspect is ShuffleBlockFetcherIterator.

Review guide:
- Most of the ConnectionManager code is now in network.cm package
- ManagedBuffer is a new buffer abstraction backed by several different implementations (file segment, nio ByteBuffer, Netty ByteBuf)
- BlockTransferService is the main internal interface introduced in this PR
- NioBlockTransferService implements BlockTransferService and replaces the old BlockManagerWorker
- ShuffleBlockFetcherIterator replaces the told BlockFetcherIterator to use the new interface

TODOs that should be separate PRs:
- Implement NettyBlockTransferService
- Finalize the API/semantics for ManagedBuffer.release()

Author: Reynold Xin <rxin@apache.org>

Closes #2240 from rxin/blockTransferService and squashes the following commits:

64cd9d7 [Reynold Xin] Merge branch 'master' into blockTransferService
1dfd3d7 [Reynold Xin] Limit the length of the FileInputStream.
1332156 [Reynold Xin] Fixed style violation from refactoring.
2960c93 [Reynold Xin] Added ShuffleBlockFetcherIteratorSuite.
e29c721 [Reynold Xin] Updated comment for ShuffleBlockFetcherIterator.
8a1046e [Reynold Xin] Code review feedback:
2c6b1e1 [Reynold Xin] Removed println in test cases.
2a907e4 [Reynold Xin] Merge branch 'master' into blockTransferService-merge
07ccf0d [Reynold Xin] Added init check to CMBlockTransferService.
98c668a [Reynold Xin] Added failure handling and fixed unit tests.
ae05fcd [Reynold Xin] Updated tests, although DistributedSuite is hanging.
d8d595c [Reynold Xin] Merge branch 'master' of github.com:apache/spark into blockTransferService
9ef279c [Reynold Xin] Initial refactoring to move ConnectionManager to use the BlockTransferService.
2014-09-08 15:59:20 -07:00
Sandy Ryza 16a73c2473 SPARK-2978. Transformation with MR shuffle semantics
I didn't add this to the transformations list in the docs because it's kind of obscure, but would be happy to do so if others think it would be helpful.

Author: Sandy Ryza <sandy@cloudera.com>

Closes #2274 from sryza/sandy-spark-2978 and squashes the following commits:

4a5332a [Sandy Ryza] Fix Java test
c04b447 [Sandy Ryza] Fix Python doc and add back deleted code
433ad5b [Sandy Ryza] Add Java test
4c25a54 [Sandy Ryza] Add s at the end and a couple other fixes
9b0ba99 [Sandy Ryza] Fix compilation
36e0571 [Sandy Ryza] Fix import ordering
48c12c2 [Sandy Ryza] Add Java version and additional doc
e5381cd [Sandy Ryza] Fix python style warnings
f147634 [Sandy Ryza] SPARK-2978. Transformation with MR shuffle semantics
2014-09-08 11:20:00 -07:00
Reynold Xin f25bbbdb3a [SPARK-3280] Made sort-based shuffle the default implementation
Sort-based shuffle has lower memory usage and seems to outperform hash-based in almost all of our testing.

Author: Reynold Xin <rxin@apache.org>

Closes #2178 from rxin/sort-shuffle and squashes the following commits:

713d341 [Reynold Xin] Fixed test failures by setting spark.shuffle.compress to the same value as spark.shuffle.spill.compress.
85165e6 [Reynold Xin] Fixed a comment typo.
aa0d372 [Reynold Xin] [SPARK-3280] Made sort-based shuffle the default implementation
2014-09-07 20:42:07 -07:00
Eric Liang 6754570d83 [SPARK-3394] [SQL] Fix crash in TakeOrdered when limit is 0
This resolves https://issues.apache.org/jira/browse/SPARK-3394

Author: Eric Liang <ekl@google.com>

Closes #2264 from ericl/spark-3394 and squashes the following commits:

c87355b [Eric Liang] refactor
bfb6140 [Eric Liang] change RDD takeOrdered instead
7a51528 [Eric Liang] fix takeordered when limit = 0
2014-09-07 17:57:59 -07:00
Reynold Xin 3fb57a0ab3 [SPARK-3353] parent stage should have lower stage id.
Previously parent stages had higher stage id, but parent stages are executed first. This pull request changes the behavior so parent stages would have lower stage id.

For example, command:
```scala
sc.parallelize(1 to 10).map(x=>(x,x)).reduceByKey(_+_).count
```
breaks down into 2 stages.

The old web UI:
![screen shot 2014-09-04 at 12 42 44 am](https://cloud.githubusercontent.com/assets/323388/4146177/60fb4f42-3407-11e4-819f-853eb0e22b25.png)

Web UI with this patch:
![screen shot 2014-09-04 at 12 44 55 am](https://cloud.githubusercontent.com/assets/323388/4146178/62e08e62-3407-11e4-867b-a36b10534464.png)

Author: Reynold Xin <rxin@apache.org>

Closes #2273 from rxin/lower-stage-id and squashes the following commits:

abbb4c6 [Reynold Xin] Fixed SparkListenerSuite.
0e02379 [Reynold Xin] Updated DAGSchedulerSuite.
54ccea3 [Reynold Xin] [SPARK-3353] parent stage should have lower stage id.
2014-09-06 19:06:30 -07:00
GuoQiang Li 21a1e1bb89 [SPARK-3273][SPARK-3301]We should read the version information from the same place
Author: GuoQiang Li <witgo@qq.com>

Closes #2175 from witgo/SPARK-3273 and squashes the following commits:

cf9c65a [GuoQiang Li] We should read the version information from the same place
2a44e2f [GuoQiang Li] The spark version in the welcome message of pyspark is not correct
2014-09-06 15:08:43 -07:00
Kousuke Saruta 4bba10c41a [SPARK-3233] Executor never stop its SparnEnv, BlockManager, ConnectionManager etc.
Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #2138 from sarutak/SPARK-3233 and squashes the following commits:

c0205b7 [Kousuke Saruta] Merge branch 'SPARK-3233' of github.com:sarutak/spark into SPARK-3233
064679d [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3233
d3005fd [Kousuke Saruta] Modified Class definition format of BlockManagerMaster
039b747 [Kousuke Saruta] Modified style
889e2d1 [Kousuke Saruta] Modified BlockManagerMaster to be able to be past isDriver flag
4da8535 [Kousuke Saruta] Modified BlockManagerMaster#stop to send StopBlockManagerMaster message when sender is Driver
6518c3a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3233
d5ab19a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3233
6bce25c [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3233
6058a58 [Kousuke Saruta] Modified Executor not to invoke SparkEnv#stop in local mode
e5ad9d3 [Kousuke Saruta] Modified Executor to stop SparnEnv at the end of itself
2014-09-03 18:42:01 -07:00
Marcelo Vanzin f2b5b619a9 [SPARK-3388] Expose aplication ID in ApplicationStart event, use it in history server.
This change exposes the application ID generated by the Spark Master, Mesos or Yarn
via the SparkListenerApplicationStart event. It then uses that information to expose the
application via its ID in the history server, instead of using the internal directory name
generated by the event logger as an application id. This allows someone who knows
the application ID to easily figure out the URL for the application's entry in the HS, aside
from looking better.

In Yarn mode, this is used to generate a direct link from the RM application list to the
Spark history server entry (thus providing a fix for SPARK-2150).

Note this sort of assumes that the different managers will generate app ids that are
sufficiently different from each other that clashes will not occur.

Author: Marcelo Vanzin <vanzin@cloudera.com>

This patch had conflicts when merged, resolved by
Committer: Andrew Or <andrewor14@gmail.com>

Closes #1218 from vanzin/yarn-hs-link-2 and squashes the following commits:

2d19f3c [Marcelo Vanzin] Review feedback.
6706d3a [Marcelo Vanzin] Implement applicationId() in base classes.
56fe42e [Marcelo Vanzin] Fix cluster mode history address, plus a cleanup.
44112a8 [Marcelo Vanzin] Merge branch 'master' into yarn-hs-link-2
8278316 [Marcelo Vanzin] Merge branch 'master' into yarn-hs-link-2
a86bbcf [Marcelo Vanzin] Merge branch 'master' into yarn-hs-link-2
a0056e6 [Marcelo Vanzin] Unbreak test.
4b10cfd [Marcelo Vanzin] Merge branch 'master' into yarn-hs-link-2
cb0cab2 [Marcelo Vanzin] Merge branch 'master' into yarn-hs-link-2
25f2826 [Marcelo Vanzin] Add MIMA excludes.
f0ba90f [Marcelo Vanzin] Use BufferedIterator.
c90a08d [Marcelo Vanzin] Remove unused code.
3f8ec66 [Marcelo Vanzin] Review feedback.
21aa71b [Marcelo Vanzin] Fix JSON test.
b022bae [Marcelo Vanzin] Undo SparkContext cleanup.
c6d7478 [Marcelo Vanzin] Merge branch 'master' into yarn-hs-link-2
4e3483f [Marcelo Vanzin] Fix test.
57517b8 [Marcelo Vanzin] Review feedback. Mostly, more consistent use of Scala's Option.
311e49d [Marcelo Vanzin] Merge branch 'master' into yarn-hs-link-2
d35d86f [Marcelo Vanzin] Fix yarn backend after rebase.
36dc362 [Marcelo Vanzin] Don't use Iterator::takeWhile().
0afd696 [Marcelo Vanzin] Wait until master responds before returning from start().
abc4697 [Marcelo Vanzin] Make FsHistoryProvider keep a map of applications by id.
26b266e [Marcelo Vanzin] Use Mesos framework ID as Spark application ID.
b3f3664 [Marcelo Vanzin] [yarn] Make the RM link point to the app direcly in the HS.
2fb7de4 [Marcelo Vanzin] Expose the application ID in the ApplicationStart event.
ed10348 [Marcelo Vanzin] Expose application id to spark context.
2014-09-03 14:57:38 -07:00
Marcelo Vanzin ccc69e26ec [SPARK-2845] Add timestamps to block manager events.
These are not used by the UI but are useful when analysing the
logs from a spark job.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #654 from vanzin/bm-event-tstamp and squashes the following commits:

d5d6e66 [Marcelo Vanzin] Fix tests.
ec06218 [Marcelo Vanzin] Review feedback.
f134dbc [Marcelo Vanzin] Merge branch 'master' into bm-event-tstamp
b495b7c [Marcelo Vanzin] Merge branch 'master' into bm-event-tstamp
7d2fe9e [Marcelo Vanzin] Review feedback.
d6f381c [Marcelo Vanzin] Update tests added after patch was created.
45e3bf8 [Marcelo Vanzin] Fix unit test after merge.
b37a10f [Marcelo Vanzin] Use === in test assertions.
ef72824 [Marcelo Vanzin] Handle backwards compatibility with 1.0.0.
aca1151 [Marcelo Vanzin] Fix unit test to check new fields.
efdda8e [Marcelo Vanzin] Add timestamps to block manager events.
2014-09-03 14:47:11 -07:00
Reynold Xin db160676c5 [SPARK-3135] Avoid extra mem copy in TorrentBroadcast via ByteArrayChunkOutputStream
This also enables supporting broadcast variables larger than 2G.

Author: Reynold Xin <rxin@apache.org>

Closes #2054 from rxin/ByteArrayChunkOutputStream and squashes the following commits:

618d9c8 [Reynold Xin] Code review.
93f5a51 [Reynold Xin] Added comments.
ee88e73 [Reynold Xin] to -> until
bbd1cb1 [Reynold Xin] Renamed a variable.
36f4d01 [Reynold Xin] Sort imports.
8f1a8eb [Reynold Xin] [SPARK-3135] Created ByteArrayChunkOutputStream and used it to avoid memory copy in TorrentBroadcast.
2014-09-01 20:32:31 -07:00
scwf 725715cbf3 [SPARK-3010] fix redundant conditional
https://issues.apache.org/jira/browse/SPARK-3010

this pr is to fix redundant conditional in spark, such as
1.
private[spark] def codegenEnabled: Boolean =
if (getConf(CODEGEN_ENABLED, "false") == "true") true else false
2.
x => if (x == 2) true else false
...

Author: scwf <wangfei1@huawei.com>
Author: wangfei <wangfei_hello@126.com>

Closes #1992 from scwf/condition and squashes the following commits:

b2a044a [scwf] merge SecurityManager
e16239c [scwf] fix confilct
6811401 [scwf] fix merge confilct
0824df4 [scwf] Merge branch 'master' of https://github.com/apache/spark into patch-4
e274515 [scwf] fix redundant conditions
d032bf9 [wangfei] [SQL]Excess judgment
2014-08-31 14:02:11 -07:00
Marcelo Vanzin b6cf134817 [SPARK-2889] Create Hadoop config objects consistently.
Different places in the code were instantiating Configuration / YarnConfiguration objects in different ways. This could lead to confusion for people who actually expected "spark.hadoop.*" options to end up in the configs used by Spark code, since that would only happen for the SparkContext's config.

This change modifies most places to use SparkHadoopUtil to initialize configs, and make that method do the translation that previously was only done inside SparkContext.

The places that were not changed fall in one of the following categories:
- Test code where this doesn't really matter
- Places deep in the code where plumbing SparkConf would be too difficult for very little gain
- Default values for arguments - since the caller can provide their own config in that case

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #1843 from vanzin/SPARK-2889 and squashes the following commits:

52daf35 [Marcelo Vanzin] Merge branch 'master' into SPARK-2889
f179013 [Marcelo Vanzin] Merge branch 'master' into SPARK-2889
51e71cf [Marcelo Vanzin] Add test to ensure that overriding Yarn configs works.
53f9506 [Marcelo Vanzin] Add DeveloperApi annotation.
3d345cb [Marcelo Vanzin] Restore old method for backwards compat.
fc45067 [Marcelo Vanzin] Merge branch 'master' into SPARK-2889
0ac3fdf [Marcelo Vanzin] Merge branch 'master' into SPARK-2889
3f26760 [Marcelo Vanzin] Compilation fix.
f16cadd [Marcelo Vanzin] Initialize config in SparkHadoopUtil.
b8ab173 [Marcelo Vanzin] Update Utils API to take a Configuration argument.
1e7003f [Marcelo Vanzin] Replace explicit Configuration instantiation with SparkHadoopUtil.
2014-08-30 14:48:07 -07:00
Raymond Liu acea92806c [SPARK-2288] Hide ShuffleBlockManager behind ShuffleManager
By Hiding the shuffleblockmanager behind Shufflemanager, we decouple the shuffle data's block mapping management work from Diskblockmananger. This give a more clear interface and more easy for other shuffle manager to implement their own block management logic. the jira ticket have more details.

Author: Raymond Liu <raymond.liu@intel.com>

Closes #1241 from colorant/shuffle and squashes the following commits:

0e01ae3 [Raymond Liu] Move ShuffleBlockmanager behind shuffleManager
2014-08-29 23:05:18 -07:00
Reynold Xin 665e71d14d [SPARK-1912] Lazily initialize buffers for local shuffle blocks.
This is a simplified fix for SPARK-1912.

Author: Reynold Xin <rxin@apache.org>

Closes #2179 from rxin/SPARK-1912 and squashes the following commits:

b2f0e9e [Reynold Xin] Fix unit tests.
a8eddfe [Reynold Xin] [SPARK-1912] Lazily initialize buffers for local shuffle blocks.
2014-08-28 19:00:40 -07:00
Andrew Or a46b8f2d71 [SPARK-3277] Fix external spilling with LZ4 assertion error
**Summary of the changes**

The bulk of this PR is comprised of tests and documentation; the actual fix is really just adding 1 line of code (see `BlockObjectWriter.scala`). We currently do not run the `External*` test suites with different compression codecs, and this would have caught the bug reported in [SPARK-3277](https://issues.apache.org/jira/browse/SPARK-3277). This PR extends the existing code to test spilling using all compression codecs known to Spark, including `LZ4`.

**The bug itself**

In `DiskBlockObjectWriter`, we only report the shuffle bytes written before we close the streams. With `LZ4`, all the bytes written reported by our metrics were 0 because `flush()` was not taking effect for some reason. In general, compression codecs may write additional bytes to the file after we call `close()`, and so we must also capture those bytes in our shuffle write metrics.

Thanks mridulm and pwendell for help with debugging.

Author: Andrew Or <andrewor14@gmail.com>
Author: Patrick Wendell <pwendell@gmail.com>

Closes #2187 from andrewor14/fix-lz4-spilling and squashes the following commits:

1b54bdc [Andrew Or] Speed up tests by not compressing everything
1c4624e [Andrew Or] Merge branch 'master' of github.com:apache/spark into fix-lz4-spilling
6b2e7d1 [Andrew Or] Fix compilation error
92e251b [Patrick Wendell] Better documentation for BlockObjectWriter.
a1ad536 [Andrew Or] Fix tests
089593f [Andrew Or] Actually fix SPARK-3277 (tests still fail)
4bbcf68 [Andrew Or] Update tests to actually test all compression codecs
b264a84 [Andrew Or] ExternalAppendOnlyMapSuite code style fixes (minor)
1bfa743 [Andrew Or] Add more information to assert for better debugging
2014-08-28 17:05:21 -07:00
Reynold Xin be53c54b5c [SPARK-3281] Remove Netty specific code in BlockManager / shuffle
Netty functionality will be added back in subsequent PRs by using the BlockTransferService interface.

Author: Reynold Xin <rxin@apache.org>

Closes #2181 from rxin/SPARK-3281 and squashes the following commits:

5494b0e [Reynold Xin] Fix extra port.
ff6d1e1 [Reynold Xin] [SPARK-3281] Remove Netty specific code in BlockManager.
2014-08-28 14:08:07 -07:00
uncleGen d8298c46b7 [SPARK-3170][CORE][BUG]:RDD info loss in "StorageTab" and "ExecutorTab"
compeleted stage only need to remove its own partitions that are no longer cached. However, "StorageTab" may lost some rdds which are cached actually. Not only in "StorageTab", "ExectutorTab" may also lose some rdd info which have been overwritten by last rdd in a same task.
1. "StorageTab": when multiple stages run simultaneously, completed stage will remove rdd info which belong to other stages that are still running.
2. "ExectutorTab": taskcontext may lose some "updatedBlocks" info of  rdds  in a dependency chain. Like the following example:
         val r1 = sc.paralize(..).cache()
         val r2 = r1.map(...).cache()
         val n = r2.count()

When count the r2, r1 and r2 will be cached finally. So in CacheManager.getOrCompute, the taskcontext should contain "updatedBlocks" of r1 and r2. Currently, the "updatedBlocks" only contain the info of r2.

Author: uncleGen <hustyugm@gmail.com>

Closes #2131 from uncleGen/master_ui_fix and squashes the following commits:

a6a8a0b [uncleGen] fix some coding style
3a1bc15 [uncleGen] fix some error in unit test
56ea488 [uncleGen] there's some line too long
c82ba82 [uncleGen] Bug Fix: RDD info loss in "StorageTab" and "ExecutorTab"
2014-08-27 10:33:01 -07:00
Tathagata Das 3e2864e404 [SPARK-3139] Made ContextCleaner to not block on shuffles
As a workaround for SPARK-3015, the ContextCleaner was made "blocking", that is, it cleaned items one-by-one. But shuffles can take a long time to be deleted. Given that the RC for 1.1 is imminent, this PR makes a narrow change in the context cleaner - not wait for shuffle cleanups to complete. Also it changes the error messages on failure to delete to be milder warnings, as exceptions in the delete code path for one item does not really stop the actual functioning of the system.

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

Closes #2143 from tdas/cleaner-shuffle-fix and squashes the following commits:

9c84202 [Tathagata Das] Restoring default blocking behavior in ContextCleanerSuite, and added docs to identify that spark.cleaner.referenceTracking.blocking does not control shuffle.
2181329 [Tathagata Das] Mark shuffle cleanup as non-blocking.
e337cc2 [Tathagata Das] Changed semantics based on PR comments.
387b578 [Tathagata Das] Made ContextCleaner to not block on shuffles
2014-08-27 00:13:38 -07:00
Reynold Xin bf719056b7 [SPARK-3224] FetchFailed reduce stages should only show up once in failed stages (in UI)
This is a HOTFIX for 1.1.

Author: Reynold Xin <rxin@apache.org>
Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #2127 from rxin/SPARK-3224 and squashes the following commits:

effb1ce [Reynold Xin] Move log message.
49282b3 [Reynold Xin] Kay's feedback.
3f01847 [Reynold Xin] Merge pull request #2 from kayousterhout/SPARK-3224
796d282 [Kay Ousterhout] Added unit test for SPARK-3224
3d3d356 [Reynold Xin] Remove map output loc even for repeated FetchFaileds.
1dd3eb5 [Reynold Xin] [SPARK-3224] FetchFailed reduce stages should only show up once in the failed stages UI.
2014-08-26 22:12:37 -07:00
Reynold Xin fb60bec34e [SPARK-2298] Encode stage attempt in SparkListener & UI.
Simple way to reproduce this in the UI:

```scala
val f = new java.io.File("/tmp/test")
f.delete()
sc.parallelize(1 to 2, 2).map(x => (x,x )).repartition(3).mapPartitionsWithContext { case (context, iter) =>
  if (context.partitionId == 0) {
    val f = new java.io.File("/tmp/test")
    if (!f.exists) {
      f.mkdir()
      System.exit(0);
    }
  }
  iter
}.count()
```

Author: Reynold Xin <rxin@apache.org>

Closes #1545 from rxin/stage-attempt and squashes the following commits:

3ee1d2a [Reynold Xin] - Rename attempt to retry in UI. - Properly report stage failure in FetchFailed.
40a6bd5 [Reynold Xin] Updated test suites.
c414c36 [Reynold Xin] Fixed the hanging in JobCancellationSuite.
b3e2eed [Reynold Xin] Oops previous code didn't compile.
0f36075 [Reynold Xin] Mark unknown stage attempt with id -1 and drop that in JobProgressListener.
6c08b07 [Reynold Xin] Addressed code review feedback.
4e5faa2 [Reynold Xin] [SPARK-2298] Encode stage attempt in SparkListener & UI.
2014-08-20 15:37:27 -07:00
Josh Rosen ebcb94f701 [SPARK-2974] [SPARK-2975] Fix two bugs related to spark.local.dirs
This PR fixes two bugs related to `spark.local.dirs` and `SPARK_LOCAL_DIRS`, one where `Utils.getLocalDir()` might return an invalid directory (SPARK-2974) and another where the `SPARK_LOCAL_DIRS` override didn't affect the driver, which could cause problems when running tasks in local mode (SPARK-2975).

This patch fixes both issues: the new `Utils.getOrCreateLocalRootDirs(conf: SparkConf)` utility method manages the creation of local directories and handles the precedence among the different configuration options, so we should see the same behavior whether we're running in local mode or on a worker.

It's kind of a pain to mock out environment variables in tests (no easy way to mock System.getenv), so I added a `private[spark]` method to SparkConf for accessing environment variables (by default, it just delegates to System.getenv).  By subclassing SparkConf and overriding this method, we can mock out SPARK_LOCAL_DIRS in tests.

I also fixed a typo in PySpark where we used `SPARK_LOCAL_DIR` instead of `SPARK_LOCAL_DIRS` (I think this was technically innocuous, but it seemed worth fixing).

Author: Josh Rosen <joshrosen@apache.org>

Closes #2002 from JoshRosen/local-dirs and squashes the following commits:

efad8c6 [Josh Rosen] Address review comments:
1dec709 [Josh Rosen] Minor updates to Javadocs.
7f36999 [Josh Rosen] Use env vars to detect if running in YARN container.
399ac25 [Josh Rosen] Update getLocalDir() documentation.
bb3ad89 [Josh Rosen] Remove duplicated YARN getLocalDirs() code.
3e92d44 [Josh Rosen] Move local dirs override logic into Utils; fix bugs:
b2c4736 [Josh Rosen] Add failing tests for SPARK-2974 and SPARK-2975.
007298b [Josh Rosen] Allow environment variables to be mocked in tests.
6d9259b [Josh Rosen] Fix typo in PySpark: SPARK_LOCAL_DIR should be SPARK_LOCAL_DIRS
2014-08-19 22:42:50 -07:00
Reynold Xin 8adfbc2b6b [SPARK-3119] Re-implementation of TorrentBroadcast.
This is a re-implementation of TorrentBroadcast, with the following changes:

1. Removes most of the mutable, transient state from TorrentBroadcast (e.g. totalBytes, num of blocks fetched).
2. Removes TorrentInfo and TorrentBlock
3. Replaces the BlockManager.getSingle call in readObject with a getLocal, resuling in one less RPC call to the BlockManagerMasterActor to find the location of the block.
4. Removes the metadata block, resulting in one less block to fetch.
5. Removes an extra memory copy for deserialization (by using Java's SequenceInputStream).

Basically for a regular broadcasted object with only one block, the number of RPC calls goes from 5+1 to 2+1).

Old TorrentBroadcast for object of a single block:
1 RPC to ask for location of the broadcast variable
1 RPC to ask for location of the metadata block
1 RPC to fetch the metadata block
1 RPC to ask for location of the first data block
1 RPC to fetch the first data block
1 RPC to tell the driver we put the first data block in
i.e. 5 + 1

New TorrentBroadcast for object of a single block:
1 RPC to ask for location of the first data block
1 RPC to get the first data block
1 RPC to tell the driver we put the first data block in
i.e. 2 + 1

Author: Reynold Xin <rxin@apache.org>

Closes #2030 from rxin/torrentBroadcast and squashes the following commits:

5bacb9d [Reynold Xin] Always add the object to driver's block manager.
0d8ed5b [Reynold Xin] Added getBytes to BlockManager and uses that in TorrentBroadcast.
2d6a5fb [Reynold Xin] Use putBytes/getRemoteBytes throughout.
3670f00 [Reynold Xin] Code review feedback.
c1185cd [Reynold Xin] [SPARK-3119] Re-implementation of TorrentBroadcast.
2014-08-19 22:11:13 -07:00
Reynold Xin 8b9dc99101 [SPARK-2468] Netty based block server / client module
Previous pull request (#1907) was reverted. This brings it back. Still looking into the hang.

Author: Reynold Xin <rxin@apache.org>

Closes #1971 from rxin/netty1 and squashes the following commits:

b0be96f [Reynold Xin] Added test to make sure outstandingRequests are cleaned after firing the events.
4c6d0ee [Reynold Xin] Pass callbacks cleanly.
603dce7 [Reynold Xin] Upgrade Netty to 4.0.23 to fix the DefaultFileRegion bug.
88be1d4 [Reynold Xin] Downgrade to 4.0.21 to work around a bug in writing DefaultFileRegion.
002626a [Reynold Xin] Remove netty-test-file.txt.
db6e6e0 [Reynold Xin] Revert "Revert "[SPARK-2468] Netty based block server / client module""
2014-08-19 17:40:35 -07:00
Chandan Kumar f45efbb8aa [SPARK-2862] histogram method fails on some choices of bucketCount
Author: Chandan Kumar <chandan.kumar@imaginea.com>

Closes #1787 from nrchandan/spark-2862 and squashes the following commits:

a76bbf6 [Chandan Kumar] [SPARK-2862] Fix for a broken test case and add new test cases
4211eea [Chandan Kumar] [SPARK-2862] Add Scala bug id
13854f1 [Chandan Kumar] [SPARK-2862] Use shorthand range notation to avoid Scala bug
2014-08-18 09:52:25 -07:00
Sandy Ryza df652ea02a SPARK-2900. aggregate inputBytes per stage
Author: Sandy Ryza <sandy@cloudera.com>

Closes #1826 from sryza/sandy-spark-2900 and squashes the following commits:

43f9091 [Sandy Ryza] SPARK-2900
2014-08-17 22:39:06 -07:00
Kousuke Saruta 76fa0eaf51 [SPARK-2677] BasicBlockFetchIterator#next can wait forever
Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #1632 from sarutak/SPARK-2677 and squashes the following commits:

cddbc7b [Kousuke Saruta] Removed Exception throwing when ConnectionManager#handleMessage receives ack for non-referenced message
d3bd2a8 [Kousuke Saruta] Modified configuration.md for spark.core.connection.ack.timeout
e85f88b [Kousuke Saruta] Removed useless synchronized blocks
7ed48be [Kousuke Saruta] Modified ConnectionManager to use ackTimeoutMonitor ConnectionManager-wide
9b620a6 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2677
0dd9ad3 [Kousuke Saruta] Modified typo in ConnectionManagerSuite.scala
7cbb8ca [Kousuke Saruta] Modified to match with scalastyle
8a73974 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2677
ade279a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2677
0174d6a [Kousuke Saruta] Modified ConnectionManager.scala to handle the case remote Executor cannot ack
a454239 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2677
9b7b7c1 [Kousuke Saruta] (WIP) Modifying ConnectionManager.scala
2014-08-16 14:15:58 -07:00
Josh Rosen 20fcf3d0b7 [SPARK-2977] Ensure ShuffleManager is created before ShuffleBlockManager
This is intended to fix SPARK-2977.  Before, there was an implicit ordering dependency where we needed to know the ShuffleManager implementation before creating the ShuffleBlockManager.  This patch makes that dependency explicit by adding ShuffleManager to a bunch of constructors.

I think it's a little odd for BlockManager to take a ShuffleManager only to pass it to ShuffleBlockManager without using it itself; there's an opportunity to clean this up later if we sever the circular dependencies between BlockManager and other components and pass those components to BlockManager's constructor.

Author: Josh Rosen <joshrosen@apache.org>

Closes #1976 from JoshRosen/SPARK-2977 and squashes the following commits:

a9cd1e1 [Josh Rosen] [SPARK-2977] Ensure ShuffleManager is created before ShuffleBlockManager.
2014-08-16 00:04:55 -07:00
Reynold Xin a83c7723bf [SPARK-3045] Make Serializer interface Java friendly
Author: Reynold Xin <rxin@apache.org>

Closes #1948 from rxin/kryo and squashes the following commits:

a3a80d8 [Reynold Xin] [SPARK-3046] use executor's class loader as the default serializer classloader
3d13277 [Reynold Xin] Reverted that in TestJavaSerializerImpl too.
196f3dc [Reynold Xin] Ok one more commit to revert the classloader change.
c49b50c [Reynold Xin] Removed JavaSerializer change.
afbf37d [Reynold Xin] Moved the test case also.
a2e693e [Reynold Xin] Removed the Kryo bug fix from this pull request.
c81bd6c [Reynold Xin] Use defaultClassLoader when executing user specified custom registrator.
68f261e [Reynold Xin] Added license check excludes.
0c28179 [Reynold Xin] [SPARK-3045] Make Serializer interface Java friendly [SPARK-3046] Set executor's class loader as the default serializer class loader
2014-08-15 23:12:34 -07:00
Reynold Xin cc3648774e [SPARK-3046] use executor's class loader as the default serializer classloader
The serializer is not always used in an executor thread (e.g. connection manager, broadcast), in which case the classloader might not have the user jar set, leading to corruption in deserialization.

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

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

Author: Reynold Xin <rxin@apache.org>

Closes #1972 from rxin/kryoBug and squashes the following commits:

c1c7bf0 [Reynold Xin] Made change to JavaSerializer.
7204c33 [Reynold Xin] Added imports back.
d879e67 [Reynold Xin] [SPARK-3046] use executor's class loader as the default serializer class loader.
2014-08-15 17:04:15 -07:00
Patrick Wendell fd9fcd25e9 Revert "[SPARK-2468] Netty based block server / client module"
This reverts commit 3a8b68b735.
2014-08-15 09:01:04 -07:00
Reynold Xin 3a8b68b735 [SPARK-2468] Netty based block server / client module
This is a rewrite of the original Netty module that was added about 1.5 years ago. The old code was turned off by default and didn't really work because it lacked a frame decoder (only worked with very very small blocks).

For this pull request, I tried to make the changes non-instrusive to the rest of Spark. I only added an init and shutdown to BlockManager/DiskBlockManager, and a bunch of comments to help me understand the existing code base.

Compared with the old Netty module, this one features:
- It appears to work :)
- SPARK-2941: option to specicy nio vs oio vs epoll for channel/transport. By default nio is used. (Not using Epoll yet because I have found some bugs with its implementation)
- SPARK-2943: options to specify send buf and receive buf for users who want to do hyper tuning
- SPARK-2942: io errors are reported from server to client (the protocol uses negative length to indicate error)
- SPARK-2940: fetching multiple blocks in a single request to reduce syscalls
- SPARK-2959: clients share a single thread pool
- SPARK-2990: use PooledByteBufAllocator to reduce GC (basically a Netty managed pool of buffers with jmalloc)
- SPARK-2625: added fetchWaitTime metric and fixed thread-safety issue in metrics update.
- SPARK-2367: bump Netty version to 4.0.21.Final to address an Epoll bug (https://groups.google.com/forum/#!topic/netty/O7m-HxCJpCA)

Compared with the existing communication manager, this one features:
- IMO it is substantially easier to understand
- zero-copy send for the server for on-disk blocks
- one-copy receive (due to a frame decoder)
- don't quote me on this, but I think a lot less sys calls
- SPARK-2990: use PooledByteBufAllocator to reduce GC (basically a Netty managed pool of buffers with jmalloc)
- SPARK-2941: option to specicy nio vs oio vs epoll for channel/transport. By default nio is used. (Not using Epoll yet because I have found some bugs with its implementation)
- SPARK-2943: options to specify send buf and receive buf for users who want to do hyper tuning

TODOs before it can fully replace the existing ConnectionManager, if that ever happens (most of them should probably be done in separate PRs since this needs to be turned on explicitly)
- [x] Basic test cases
- [ ] More unit/integration tests for failures
- [ ] Performance analysis
- [ ] Support client connection reuse so we don't need to keep opening new connections (not sure how useful this would be)
- [ ] Support putting blocks in addition to fetching blocks (i.e. two way transfer)
- [x] Support serving non-disk blocks
- [ ] Support SASL authentication

For a more comprehensive list, see https://issues.apache.org/jira/browse/SPARK-2468

Thanks to @coderplay for peer coding with me on a Sunday.

Author: Reynold Xin <rxin@apache.org>

Closes #1907 from rxin/netty and squashes the following commits:

f921421 [Reynold Xin] Upgrade Netty to 4.0.22.Final to fix another Epoll bug.
4b174ca [Reynold Xin] Shivaram's code review comment.
4a3dfe7 [Reynold Xin] Switched to nio for default (instead of epoll on Linux).
56bfb9d [Reynold Xin] Bump Netty version to 4.0.21.Final for some bug fixes.
b443a4b [Reynold Xin] Added debug message to help debug Jenkins failures.
57fc4d7 [Reynold Xin] Added test cases for BlockHeaderEncoder and BlockFetchingClientHandlerSuite.
22623e9 [Reynold Xin] Added exception handling and test case for BlockServerHandler and BlockFetchingClientHandler.
6550dd7 [Reynold Xin] Fixed block mgr init bug.
60c2edf [Reynold Xin] Beefed up server/client integration tests.
38d88d5 [Reynold Xin] Added missing test files.
6ce3f3c [Reynold Xin] Added some basic test cases.
47f7ce0 [Reynold Xin] Created server and client packages and moved files there.
b16f412 [Reynold Xin] Added commit count.
f13022d [Reynold Xin] Remove unused clone() in BlockFetcherIterator.
c57d68c [Reynold Xin] Added back missing files.
842dfa7 [Reynold Xin] Made everything work with proper reference counting.
3fae001 [Reynold Xin] Connected the new netty network module with rest of Spark.
1a8f6d4 [Reynold Xin] Completed protocol documentation.
2951478 [Reynold Xin] New Netty implementation.
cc7843d [Reynold Xin] Basic skeleton.
2014-08-14 19:01:33 -07:00
Reynold Xin 655699f8b7 [SPARK-3027] TaskContext: tighten visibility and provide Java friendly callback API
Note this also passes the TaskContext itself to the TaskCompletionListener. In the future we can mark TaskContext with the exception object if exception occurs during task execution.

Author: Reynold Xin <rxin@apache.org>

Closes #1938 from rxin/TaskContext and squashes the following commits:

145de43 [Reynold Xin] Added JavaTaskCompletionListenerImpl for Java API friendly guarantee.
f435ea5 [Reynold Xin] Added license header for TaskCompletionListener.
dc4ed27 [Reynold Xin] [SPARK-3027] TaskContext: tighten the visibility and provide Java friendly callback API
2014-08-14 18:37:02 -07:00
Graham Dennis 6b8de0e36c SPARK-2893: Do not swallow Exceptions when running a custom kryo registrator
The previous behaviour of swallowing ClassNotFound exceptions when running a custom Kryo registrator could lead to difficult to debug problems later on at serialisation / deserialisation time, see SPARK-2878.  Instead it is better to fail fast.

Added test case.

Author: Graham Dennis <graham.dennis@gmail.com>

Closes #1827 from GrahamDennis/feature/spark-2893 and squashes the following commits:

fbe4cb6 [Graham Dennis] [SPARK-2878]: Update the test case to match the updated exception message
65e53c5 [Graham Dennis] [SPARK-2893]: Improve message when a spark.kryo.registrator fails.
f480d85 [Graham Dennis] [SPARK-2893] Fix typo.
b59d2c2 [Graham Dennis] SPARK-2893: Do not swallow Exceptions when running a custom spark.kryo.registrator
2014-08-14 02:24:18 -07:00
Aaron Davidson d069c5d9d2 [SPARK-3029] Disable local execution of Spark jobs by default
Currently, local execution of Spark jobs is only used by take(), and it can be problematic as it can load a significant amount of data onto the driver. The worst case scenarios occur if the RDD is cached (guaranteed to load whole partition), has very large elements, or the partition is just large and we apply a filter with high selectivity or computational overhead.

Additionally, jobs that run locally in this manner do not show up in the web UI, and are thus harder to track or understand what is occurring.

This PR adds a flag to disable local execution, which is turned OFF by default, with the intention of perhaps eventually removing this functionality altogether. Removing it now is a tougher proposition since it is part of the public runJob API. An alternative solution would be to limit the flag to take()/first() to avoid impacting any external users of this API, but such usage (or, at least, reliance upon the feature) is hopefully minimal.

Author: Aaron Davidson <aaron@databricks.com>

Closes #1321 from aarondav/allowlocal and squashes the following commits:

136b253 [Aaron Davidson] Fix DAGSchedulerSuite
5599d55 [Aaron Davidson] [RFC] Disable local execution of Spark jobs by default
2014-08-14 01:37:38 -07:00
Reynold Xin 676f98289d [SPARK-2953] Allow using short names for io compression codecs
Instead of requiring "org.apache.spark.io.LZ4CompressionCodec", it is easier for users if Spark just accepts "lz4", "lzf", "snappy".

Author: Reynold Xin <rxin@apache.org>

Closes #1873 from rxin/compressionCodecShortForm and squashes the following commits:

9f50962 [Reynold Xin] Specify short-form compression codec names first.
63f78ee [Reynold Xin] Updated configuration documentation.
47b3848 [Reynold Xin] [SPARK-2953] Allow using short names for io compression codecs
2014-08-12 22:50:29 -07:00
Josh Rosen 7712e724ad [SPARK-2931] In TaskSetManager, reset currentLocalityIndex after recomputing locality levels
This addresses SPARK-2931, a bug where getAllowedLocalityLevel() could throw ArrayIndexOutOfBoundsException.  The fix here is to reset currentLocalityIndex after recomputing the locality levels.

Thanks to kayousterhout, mridulm, and lirui-intel for helping me to debug this.

Author: Josh Rosen <joshrosen@apache.org>

Closes #1896 from JoshRosen/SPARK-2931 and squashes the following commits:

48b60b5 [Josh Rosen] Move FakeRackUtil.cleanUp() info beforeEach().
6fec474 [Josh Rosen] Set currentLocalityIndex after recomputing locality levels.
9384897 [Josh Rosen] Update SPARK-2931 test to reflect changes in 63bdb1f41b.
9ecd455 [Josh Rosen] Apply @mridulm's patch for reproducing SPARK-2931.
2014-08-11 19:15:01 -07:00
Doris Xin b715aa0c80 [SPARK-2937] Separate out samplyByKeyExact as its own API in PairRDDFunction
To enable Python consistency and `Experimental` label of the `sampleByKeyExact` API.

Author: Doris Xin <doris.s.xin@gmail.com>
Author: Xiangrui Meng <meng@databricks.com>

Closes #1866 from dorx/stratified and squashes the following commits:

0ad97b2 [Doris Xin] reviewer comments.
2948aae [Doris Xin] remove unrelated changes
e990325 [Doris Xin] Merge branch 'master' into stratified
555a3f9 [Doris Xin] separate out sampleByKeyExact as its own API
616e55c [Doris Xin] merge master
245439e [Doris Xin] moved minSamplingRate to getUpperBound
eaf5771 [Doris Xin] bug fixes.
17a381b [Doris Xin] fixed a merge issue and a failed unit
ea7d27f [Doris Xin] merge master
b223529 [Xiangrui Meng] use approx bounds for poisson fix poisson mean for waitlisting add unit tests for Java
b3013a4 [Xiangrui Meng] move math3 back to test scope
eecee5f [Doris Xin] Merge branch 'master' into stratified
f4c21f3 [Doris Xin] Reviewer comments
a10e68d [Doris Xin] style fix
a2bf756 [Doris Xin] Merge branch 'master' into stratified
680b677 [Doris Xin] use mapPartitionWithIndex instead
9884a9f [Doris Xin] style fix
bbfb8c9 [Doris Xin] Merge branch 'master' into stratified
ee9d260 [Doris Xin] addressed reviewer comments
6b5b10b [Doris Xin] Merge branch 'master' into stratified
254e03c [Doris Xin] minor fixes and Java API.
4ad516b [Doris Xin] remove unused imports from PairRDDFunctions
bd9dc6e [Doris Xin] unit bug and style violation fixed
1fe1cff [Doris Xin] Changed fractionByKey to a map to enable arg check
944a10c [Doris Xin] [SPARK-2145] Add lower bound on sampling rate
0214a76 [Doris Xin] cleanUp
90d94c0 [Doris Xin] merge master
9e74ab5 [Doris Xin] Separated out most of the logic in sampleByKey
7327611 [Doris Xin] merge master
50581fc [Doris Xin] added a TODO for logging in python
46f6c8c [Doris Xin] fixed the NPE caused by closures being cleaned before being passed into the aggregate function
7e1a481 [Doris Xin] changed the permission on SamplingUtil
1d413ce [Doris Xin] fixed checkstyle issues
9ee94ee [Doris Xin] [SPARK-2082] stratified sampling in PairRDDFunctions that guarantees exact sample size
e3fd6a6 [Doris Xin] Merge branch 'master' into takeSample
7cab53a [Doris Xin] fixed import bug in rdd.py
ffea61a [Doris Xin] SPARK-1939: Refactor takeSample method in RDD
1441977 [Doris Xin] SPARK-1939 Refactor takeSample method in RDD to use ScaSRS
2014-08-10 16:31:07 -07:00
GuoQiang Li ec79063fad [SPARK-2897][SPARK-2920]TorrentBroadcast does use the serializer class specified in the spark option "spark.serializer"
Author: GuoQiang Li <witgo@qq.com>

Closes #1836 from witgo/SPARK-2897 and squashes the following commits:

23cdc5b [GuoQiang Li] review commit
ada4fba [GuoQiang Li] TorrentBroadcast does not support broadcast compression
fb91792 [GuoQiang Li] org.apache.spark.broadcast.TorrentBroadcast does use the serializer class specified in the spark option "spark.serializer"
2014-08-08 16:57:26 -07:00
Erik Erlandson 9a54de16ed [SPARK-2911]: provide rdd.parent[T](j) to obtain jth parent RDD
Author: Erik Erlandson <eerlands@redhat.com>

Closes #1841 from erikerlandson/spark-2911-pr and squashes the following commits:

4699e2f [Erik Erlandson] [SPARK-2911]: provide rdd.parent[T](j) to obtain jth parent RDD
2014-08-07 23:45:16 -07:00
Sandy Ryza 4c51098f32 SPARK-2565. Update ShuffleReadMetrics as blocks are fetched
Author: Sandy Ryza <sandy@cloudera.com>

Closes #1507 from sryza/sandy-spark-2565 and squashes the following commits:

74dad41 [Sandy Ryza] SPARK-2565. Update ShuffleReadMetrics as blocks are fetched
2014-08-07 18:09:19 -07:00
Matei Zaharia 6906b69cf5 SPARK-2787: Make sort-based shuffle write files directly when there's no sorting/aggregation and # partitions is small
As described in https://issues.apache.org/jira/browse/SPARK-2787, right now sort-based shuffle is more expensive than hash-based for map operations that do no partial aggregation or sorting, such as groupByKey. This is because it has to serialize each data item twice (once when spilling to intermediate files, and then again when merging these files object-by-object). This patch adds a code path to just write separate files directly if the # of output partitions is small, and concatenate them at the end to produce a sorted file.

On the unit test side, I added some tests that force or don't force this bypass path to be used, and checked that our tests for other features (e.g. all the operations) cover both cases.

Author: Matei Zaharia <matei@databricks.com>

Closes #1799 from mateiz/SPARK-2787 and squashes the following commits:

88cf26a [Matei Zaharia] Fix rebase
10233af [Matei Zaharia] Review comments
398cb95 [Matei Zaharia] Fix looking up shuffle manager in conf
ca3efd9 [Matei Zaharia] Add docs for shuffle manager properties, and allow short names for them
d0ae3c5 [Matei Zaharia] Fix some comments
90d084f [Matei Zaharia] Add code path to bypass merge-sort in ExternalSorter, and tests
31e5d7c [Matei Zaharia] Move existing logic for writing partitioned files into ExternalSorter
2014-08-07 18:04:49 -07:00
Davies Liu ffd1f59a62 [SPARK-2887] fix bug of countApproxDistinct() when have more than one partition
fix bug of countApproxDistinct() when have more than one partition

Author: Davies Liu <davies.liu@gmail.com>

Closes #1812 from davies/approx and squashes the following commits:

bf757ce [Davies Liu] fix bug of countApproxDistinct() when have more than one partition
2014-08-06 21:22:13 -07:00
Kousuke Saruta 17caae48b3 [SPARK-2583] ConnectionManager error reporting
This patch modifies the ConnectionManager so that error messages are sent in reply when uncaught exceptions occur during message processing.  This prevents message senders from hanging while waiting for an acknowledgment if the remote message processing failed.

This is an updated version of sarutak's PR, #1490.  The main change is to use Futures / Promises to signal errors.

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>
Author: Josh Rosen <joshrosen@apache.org>

Closes #1758 from JoshRosen/connection-manager-fixes and squashes the following commits:

68620cb [Josh Rosen] Fix test in BlockFetcherIteratorSuite:
83673de [Josh Rosen] Error ACKs should trigger IOExceptions, so catch only those exceptions in the test.
b8bb4d4 [Josh Rosen] Fix manager.id vs managerServer.id typo that broke security tests.
659521f [Josh Rosen] Include previous exception when throwing new one
a2f745c [Josh Rosen] Remove sendMessageReliablySync; callers can wait themselves.
c01c450 [Josh Rosen] Return Try[Message] from sendMessageReliablySync.
f1cd1bb [Josh Rosen] Clean up @sarutak's PR #1490 for [SPARK-2583]: ConnectionManager error reporting
7399c6b [Josh Rosen] Merge remote-tracking branch 'origin/pr/1490' into connection-manager-fixes
ee91bb7 [Kousuke Saruta] Modified BufferMessage.scala to keep the spark code style
9dfd0d8 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2583
e7d9aa6 [Kousuke Saruta] rebase to master
326a17f [Kousuke Saruta] Add test cases to ConnectionManagerSuite.scala for SPARK-2583
2a18d6b [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2583
22d7ebd [Kousuke Saruta] Add test cases to BlockManagerSuite for SPARK-2583
e579302 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2583
281589c [Kousuke Saruta] Add a test case to BlockFetcherIteratorSuite.scala for fetching block from remote from successfully
0654128 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2583
ffaa83d [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2583
12d3de8 [Kousuke Saruta] Added BlockFetcherIteratorSuite.scala
4117b8f [Kousuke Saruta] Modified ConnectionManager to be alble to handle error during processing message
717c9c3 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2583
6635467 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2583
e2b8c4a [Kousuke Saruta] Modify to propagete error using ConnectionManager
2014-08-06 17:27:55 -07:00
Sandy Ryza 4e98236442 SPARK-2566. Update ShuffleWriteMetrics incrementally
I haven't tested this out on a cluster yet, but wanted to make sure the approach (passing ShuffleWriteMetrics down to DiskBlockObjectWriter) was ok

Author: Sandy Ryza <sandy@cloudera.com>

Closes #1481 from sryza/sandy-spark-2566 and squashes the following commits:

8090d88 [Sandy Ryza] Fix ExternalSorter
b2a62ed [Sandy Ryza] Fix more test failures
8be6218 [Sandy Ryza] Fix test failures and mark a couple variables private
c5e68e5 [Sandy Ryza] SPARK-2566. Update ShuffleWriteMetrics incrementally
2014-08-06 13:10:33 -07:00
Cheng Lian a6cd31108f [SPARK-2678][Core][SQL] A workaround for SPARK-2678
JIRA issues:

- Main: [SPARK-2678](https://issues.apache.org/jira/browse/SPARK-2678)
- Related: [SPARK-2874](https://issues.apache.org/jira/browse/SPARK-2874)

Related PR:

- #1715

This PR is both a fix for SPARK-2874 and a workaround for SPARK-2678. Fixing SPARK-2678 completely requires some API level changes that need further discussion, and we decided not to include it in Spark 1.1 release. As currently SPARK-2678 only affects Spark SQL scripts, this workaround is enough for Spark 1.1. Command line option handling logic in bash scripts looks somewhat dirty and duplicated, but it helps to provide a cleaner user interface as well as retain full downward compatibility for now.

Author: Cheng Lian <lian.cs.zju@gmail.com>

Closes #1801 from liancheng/spark-2874 and squashes the following commits:

8045d7a [Cheng Lian] Make sure test suites pass
8493a9e [Cheng Lian] Using eval to retain quoted arguments
aed523f [Cheng Lian] Fixed typo in bin/spark-sql
f12a0b1 [Cheng Lian] Worked arount SPARK-2678
daee105 [Cheng Lian] Fixed usage messages of all Spark SQL related scripts
2014-08-06 12:28:35 -07:00
Andrew Or 09f7e4587b [SPARK-2157] Enable tight firewall rules for Spark
The goal of this PR is to allow users of Spark to write tight firewall rules for their clusters. This is currently not possible because Spark uses random ports in many places, notably the communication between executors and drivers. The changes in this PR are based on top of ash211's changes in #1107.

The list covered here may or may not be the complete set of port needed for Spark to operate perfectly. However, as of the latest commit there are no known sources of random ports (except in tests). I have not documented a few of the more obscure configs.

My spark-env.sh looks like this:
```
export SPARK_MASTER_PORT=6060
export SPARK_WORKER_PORT=7070
export SPARK_MASTER_WEBUI_PORT=9090
export SPARK_WORKER_WEBUI_PORT=9091
```
and my spark-defaults.conf looks like this:
```
spark.master spark://andrews-mbp:6060
spark.driver.port 5001
spark.fileserver.port 5011
spark.broadcast.port 5021
spark.replClassServer.port 5031
spark.blockManager.port 5041
spark.executor.port 5051
```

Author: Andrew Or <andrewor14@gmail.com>
Author: Andrew Ash <andrew@andrewash.com>

Closes #1777 from andrewor14/configure-ports and squashes the following commits:

621267b [Andrew Or] Merge branch 'master' of github.com:apache/spark into configure-ports
8a6b820 [Andrew Or] Use a random UI port during tests
7da0493 [Andrew Or] Fix tests
523c30e [Andrew Or] Add test for isBindCollision
b97b02a [Andrew Or] Minor fixes
c22ad00 [Andrew Or] Merge branch 'master' of github.com:apache/spark into configure-ports
93d359f [Andrew Or] Executors connect to wrong port when collision occurs
d502e5f [Andrew Or] Handle port collisions when creating Akka systems
a2dd05c [Andrew Or] Patrick's comment nit
86461e2 [Andrew Or] Remove spark.executor.env.port and spark.standalone.client.port
1d2d5c6 [Andrew Or] Fix ports for standalone cluster mode
cb3be88 [Andrew Or] Various doc fixes (broken link, format etc.)
e837cde [Andrew Or] Remove outdated TODOs
bfbab28 [Andrew Or] Merge branch 'master' of github.com:apache/spark into configure-ports
de1b207 [Andrew Or] Update docs to reflect new ports
b565079 [Andrew Or] Add spark.ports.maxRetries
2551eb2 [Andrew Or] Remove spark.worker.watcher.port
151327a [Andrew Or] Merge branch 'master' of github.com:apache/spark into configure-ports
9868358 [Andrew Or] Add a few miscellaneous ports
6016e77 [Andrew Or] Add spark.executor.port
8d836e6 [Andrew Or] Also document SPARK_{MASTER/WORKER}_WEBUI_PORT
4d9e6f3 [Andrew Or] Fix super subtle bug
3f8e51b [Andrew Or] Correct erroneous docs...
e111d08 [Andrew Or] Add names for UI services
470f38c [Andrew Or] Special case non-"Address already in use" exceptions
1d7e408 [Andrew Or] Treat 0 ports specially + return correct ConnectionManager port
ba32280 [Andrew Or] Minor fixes
6b550b0 [Andrew Or] Assorted fixes
73fbe89 [Andrew Or] Move start service logic to Utils
ec676f4 [Andrew Or] Merge branch 'SPARK-2157' of github.com:ash211/spark into configure-ports
038a579 [Andrew Ash] Trust the server start function to report the port the service started on
7c5bdc4 [Andrew Ash] Fix style issue
0347aef [Andrew Ash] Unify port fallback logic to a single place
24a4c32 [Andrew Ash] Remove type on val to match surrounding style
9e4ad96 [Andrew Ash] Reformat for style checker
5d84e0e [Andrew Ash] Document new port configuration options
066dc7a [Andrew Ash] Fix up HttpServer port increments
cad16da [Andrew Ash] Add fallover increment logic for HttpServer
c5a0568 [Andrew Ash] Fix ConnectionManager to retry with increment
b80d2fd [Andrew Ash] Make Spark's block manager port configurable
17c79bb [Andrew Ash] Add a configuration option for spark-shell's class server
f34115d [Andrew Ash] SPARK-1176 Add port configuration for HttpBroadcast
49ee29b [Andrew Ash] SPARK-1174 Add port configuration for HttpFileServer
1c0981a [Andrew Ash] Make port in HttpServer configurable
2014-08-06 00:07:40 -07:00
CodingCat 63bdb1f41b SPARK-2294: fix locality inversion bug in TaskManager
copied from original JIRA (https://issues.apache.org/jira/browse/SPARK-2294):

If an executor E is free, a task may be speculatively assigned to E when there are other tasks in the job that have not been launched (at all) yet. Similarly, a task without any locality preferences may be assigned to E when there was another NODE_LOCAL task that could have been scheduled.
This happens because TaskSchedulerImpl calls TaskSetManager.resourceOffer (which in turn calls TaskSetManager.findTask) with increasing locality levels, beginning with PROCESS_LOCAL, followed by NODE_LOCAL, and so on until the highest currently allowed level. Now, supposed NODE_LOCAL is the highest currently allowed locality level. The first time findTask is called, it will be called with max level PROCESS_LOCAL; if it cannot find any PROCESS_LOCAL tasks, it will try to schedule tasks with no locality preferences or speculative tasks. As a result, speculative tasks or tasks with no preferences may be scheduled instead of NODE_LOCAL tasks.

----

I added an additional parameter in resourceOffer and findTask, maxLocality, indicating when we should consider the tasks without locality preference

Author: CodingCat <zhunansjtu@gmail.com>

Closes #1313 from CodingCat/SPARK-2294 and squashes the following commits:

bf3f13b [CodingCat] rollback some forgotten changes
89f9bc0 [CodingCat] address matei's comments
18cae02 [CodingCat] add test case for node-local tasks
2ba6195 [CodingCat] fix failed test cases
87dd09e [CodingCat] fix style
9b9432f [CodingCat] remove hasNodeLocalOnlyTasks
fdd1573 [CodingCat] fix failed test cases
941a4fd [CodingCat] see my shocked face..........
f600085 [CodingCat] remove hasNodeLocalOnlyTasks checking
0b8a46b [CodingCat] test whether hasNodeLocalOnlyTasks affect the results
73ceda8 [CodingCat] style fix
b3a430b [CodingCat] remove fine granularity tracking for node-local only tasks
f9a2ad8 [CodingCat] simplify the logic in TaskSchedulerImpl
c8c1de4 [CodingCat] simplify the patch
be652ed [CodingCat] avoid unnecessary delay when we only have nopref tasks
dee9e22 [CodingCat] fix locality inversion bug in TaskManager by moving nopref branch
2014-08-05 23:02:58 -07:00
Patrick Wendell 74f82c71b0 SPARK-2380: Support displaying accumulator values in the web UI
This patch adds support for giving accumulators user-visible names and displaying accumulator values in the web UI. This allows users to create custom counters that can display in the UI. The current approach displays both the accumulator deltas caused by each task and a "current" value of the accumulator totals for each stage, which gets update as tasks finish.

Currently in Spark developers have been extending the `TaskMetrics` functionality to provide custom instrumentation for RDD's. This provides a potentially nicer alternative of going through the existing accumulator framework (actually `TaskMetrics` and accumulators are on an awkward collision course as we add more features to the former). The current patch demo's how we can use the feature to provide instrumentation for RDD input sizes. The nice thing about going through accumulators is that users can read the current value of the data being tracked in their programs. This could be useful to e.g. decide to short-circuit a Spark stage depending on how things are going.

![counters](https://cloud.githubusercontent.com/assets/320616/3488815/6ee7bc34-0505-11e4-84ce-e36d9886e2cf.png)

Author: Patrick Wendell <pwendell@gmail.com>

Closes #1309 from pwendell/metrics and squashes the following commits:

8815308 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into HEAD
93fbe0f [Patrick Wendell] Other minor fixes
cc43f68 [Patrick Wendell] Updating unit tests
c991b1b [Patrick Wendell] Moving some code into the Accumulators class
9a9ba3c [Patrick Wendell] More merge fixes
c5ace9e [Patrick Wendell] More merge conflicts
1da15e3 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into metrics
9860c55 [Patrick Wendell] Potential solution to posting listener events
0bb0e33 [Patrick Wendell] Remove "display" variable and assume display = name.isDefined
0ec4ac7 [Patrick Wendell] Java API's
e95bf69 [Patrick Wendell] Stash
be97261 [Patrick Wendell] Style fix
8407308 [Patrick Wendell] Removing examples in Hadoop and RDD class
64d405f [Patrick Wendell] Adding missing file
5d8b156 [Patrick Wendell] Changes based on Kay's review.
9f18bad [Patrick Wendell] Minor style changes and tests
7a63abc [Patrick Wendell] Adding Json serialization and responding to Reynold's feedback
ad85076 [Patrick Wendell] Example of using named accumulators for custom RDD metrics.
0b72660 [Patrick Wendell] Initial WIP example of supporing globally named accumulators.
2014-08-05 13:08:23 -07:00
Thomas Graves 1c5555a23d SPARK-1890 and SPARK-1891- add admin and modify acls
It was easier to combine these 2 jira since they touch many of the same places.  This pr adds the following:

- adds modify acls
- adds admin acls (list of admins/users that get added to both view and modify acls)
- modify Kill button on UI to take modify acls into account
- changes config name of spark.ui.acls.enable to spark.acls.enable since I choose poorly in original name. We keep backwards compatibility so people can still use spark.ui.acls.enable. The acls should apply to any web ui as well as any CLI interfaces.
- send view and modify acls information on to YARN so that YARN interfaces can use (yarn cli for killing applications for example).

Author: Thomas Graves <tgraves@apache.org>

Closes #1196 from tgravescs/SPARK-1890 and squashes the following commits:

8292eb1 [Thomas Graves] review comments
b92ec89 [Thomas Graves] remove unneeded variable from applistener
4c765f4 [Thomas Graves] Add in admin acls
72eb0ac [Thomas Graves] Add modify acls
2014-08-05 12:52:52 -05:00
Matei Zaharia 4fde28c206 SPARK-2711. Create a ShuffleMemoryManager to track memory for all spilling collections
This tracks memory properly if there are multiple spilling collections in the same task (which was a problem before), and also implements an algorithm that lets each thread grow up to 1 / 2N of the memory pool (where N is the number of threads) before spilling, which avoids an inefficiency with small spills we had before (some threads would spill many times at 0-1 MB because the pool was allocated elsewhere).

Author: Matei Zaharia <matei@databricks.com>

Closes #1707 from mateiz/spark-2711 and squashes the following commits:

debf75b [Matei Zaharia] Review comments
24f28f3 [Matei Zaharia] Small rename
c8f3a8b [Matei Zaharia] Update ShuffleMemoryManager to be able to partially grant requests
315e3a5 [Matei Zaharia] Some review comments
b810120 [Matei Zaharia] Create central manager to track memory for all spilling collections
2014-08-04 23:41:03 -07:00
Reynold Xin 05bf4e4aff [SPARK-2323] Exception in accumulator update should not crash DAGScheduler & SparkContext
Author: Reynold Xin <rxin@apache.org>

Closes #1772 from rxin/accumulator-dagscheduler and squashes the following commits:

6a58520 [Reynold Xin] [SPARK-2323] Exception in accumulator update should not crash DAGScheduler & SparkContext.
2014-08-04 20:39:18 -07:00
Matei Zaharia 8e7d5ba1a2 SPARK-2792. Fix reading too much or too little data from each stream in ExternalMap / Sorter
All these changes are from mridulm's work in #1609, but extracted here to fix this specific issue and make it easier to merge not 1.1. This particular set of changes is to make sure that we read exactly the right range of bytes from each spill file in EAOM: some serializers can write bytes after the last object (e.g. the TC_RESET flag in Java serialization) and that would confuse the previous code into reading it as part of the next batch. There are also improvements to cleanup to make sure files are closed.

In addition to bringing in the changes to ExternalAppendOnlyMap, I also copied them to the corresponding code in ExternalSorter and updated its test suite to test for the same issues.

Author: Matei Zaharia <matei@databricks.com>

Closes #1722 from mateiz/spark-2792 and squashes the following commits:

5d4bfb5 [Matei Zaharia] Make objectStreamReset counter count the last object written too
18fe865 [Matei Zaharia] Update docs on objectStreamReset
576ee83 [Matei Zaharia] Allow objectStreamReset to be 0
0374217 [Matei Zaharia] Remove super paranoid code to close file handles
bda37bb [Matei Zaharia] Implement Mridul's ExternalAppendOnlyMap fixes in ExternalSorter too
0d6dad7 [Matei Zaharia] Added Mridul's test changes for ExternalAppendOnlyMap
9a78e4b [Matei Zaharia] Add @mridulm's fixes to ExternalAppendOnlyMap for batch sizes
2014-08-04 12:59:18 -07:00
Andrew Or e09e18b312 [HOTFIX] Do not throw NPE if spark.test.home is not set
`spark.test.home` was introduced in #1734. This is fine for SBT but is failing maven tests. Either way it shouldn't throw an NPE.

Author: Andrew Or <andrewor14@gmail.com>

Closes #1739 from andrewor14/fix-spark-test-home and squashes the following commits:

ce2624c [Andrew Or] Do not throw NPE if spark.test.home is not set
2014-08-02 12:12:56 -07:00
Andrew Or 148af6082c [SPARK-2454] Do not ship spark home to Workers
When standalone Workers launch executors, they inherit the Spark home set by the driver. This means if the worker machines do not share the same directory structure as the driver node, the Workers will attempt to run scripts (e.g. bin/compute-classpath.sh) that do not exist locally and fail. This is a common scenario if the driver is launched from outside of the cluster.

The solution is to simply not pass the driver's Spark home to the Workers. This PR further makes an attempt to avoid overloading the usages of `spark.home`, which is now only used for setting executor Spark home on Mesos and in python.

This is based on top of #1392 and originally reported by YanTangZhai. Tested on standalone cluster.

Author: Andrew Or <andrewor14@gmail.com>

Closes #1734 from andrewor14/spark-home-reprise and squashes the following commits:

f71f391 [Andrew Or] Revert changes in python
1c2532c [Andrew Or] Merge branch 'master' of github.com:apache/spark into spark-home-reprise
188fc5d [Andrew Or] Avoid using spark.home where possible
09272b7 [Andrew Or] Always use Worker's working directory as spark home
2014-08-02 00:45:38 -07:00