Commit graph

4497 commits

Author SHA1 Message Date
zsxwing f15806a8f8 [SPARK-6602][Core] Replace direct use of Akka with Spark RPC interface - part 1
This PR replaced the following `Actor`s to `RpcEndpoint`:

1. HeartbeatReceiver
1. ExecutorActor
1. BlockManagerMasterActor
1. BlockManagerSlaveActor
1. CoarseGrainedExecutorBackend and subclasses
1. CoarseGrainedSchedulerBackend.DriverActor

This is the first PR. I will split the work of SPARK-6602 to several PRs for code review.

Author: zsxwing <zsxwing@gmail.com>

Closes #5268 from zsxwing/rpc-rewrite and squashes the following commits:

287e9f8 [zsxwing] Fix the code style
26c56b7 [zsxwing] Merge branch 'master' into rpc-rewrite
9cc825a [zsxwing] Rmove setupThreadSafeEndpoint and add ThreadSafeRpcEndpoint
30a9036 [zsxwing] Make self return null after stopping RpcEndpointRef; fix docs and error messages
705245d [zsxwing] Fix some bugs after rebasing the changes on the master
003cf80 [zsxwing] Update CoarseGrainedExecutorBackend and CoarseGrainedSchedulerBackend to use RpcEndpoint
7d0e6dc [zsxwing] Update BlockManagerSlaveActor to use RpcEndpoint
f5d6543 [zsxwing] Update BlockManagerMaster to use RpcEndpoint
30e3f9f [zsxwing] Update ExecutorActor to use RpcEndpoint
478b443 [zsxwing] Update HeartbeatReceiver to use RpcEndpoint
2015-04-04 11:52:05 -07:00
Marcelo Vanzin 14632b7942 [SPARK-6688] [core] Always use resolved URIs in EventLoggingListener.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #5340 from vanzin/SPARK-6688 and squashes the following commits:

ccfddd9 [Marcelo Vanzin] Resolve at the source.
20d2a34 [Marcelo Vanzin] [SPARK-6688] [core] Always use resolved URIs in EventLoggingListener.
2015-04-03 11:55:04 -07:00
zsxwing 88504b75ee [SPARK-6640][Core] Fix the race condition of creating HeartbeatReceiver and retrieving HeartbeatReceiver
This PR moved the code of creating `HeartbeatReceiver` above the code of creating `schedulerBackend` to resolve the race condition.

Author: zsxwing <zsxwing@gmail.com>

Closes #5306 from zsxwing/SPARK-6640 and squashes the following commits:

840399d [zsxwing] Don't send TaskScheduler through Akka
a90616a [zsxwing] Fix docs
dd202c7 [zsxwing] Fix typo
d7c250d [zsxwing] Fix the race condition of creating HeartbeatReceiver and retrieving HeartbeatReceiver
2015-04-03 11:44:27 -07:00
Ilya Ganelin 2c43ea38ee [SPARK-6492][CORE] SparkContext.stop() can deadlock when DAGSchedulerEventProcessLoop dies
I've added a timeout and retry loop around the SparkContext shutdown code that should fix this deadlock. If a SparkContext shutdown is in progress when another thread comes knocking, it will wait for 10 seconds for the lock, then fall through where the outer loop will re-submit the request.

Author: Ilya Ganelin <ilya.ganelin@capitalone.com>

Closes #5277 from ilganeli/SPARK-6492 and squashes the following commits:

8617a7e [Ilya Ganelin] Resolved merge conflict
2fbab66 [Ilya Ganelin] Added MIMA Exclude
a0e2c70 [Ilya Ganelin] Deleted stale imports
fa28ce7 [Ilya Ganelin] reverted to just having a single stopped
76fc825 [Ilya Ganelin] Updated to use atomic booleans instead of the synchronized vars
6e8a7f7 [Ilya Ganelin] Removing unecessary null check for now since i'm not fixing stop ordering yet
cdf7073 [Ilya Ganelin] [SPARK-6492] Moved stopped=true back to the start of the shutdown sequence so this can be addressed in a seperate PR
7fb795b [Ilya Ganelin] Spacing
b7a0c5c [Ilya Ganelin] Import ordering
df8224f [Ilya Ganelin] Added comment for added lock
343cb94 [Ilya Ganelin] [SPARK-6492] Added timeout/retry logic to fix a deadlock in SparkContext shutdown
2015-04-03 19:23:11 +01:00
Stephen Haberman b0d884f044 [SPARK-6560][CORE] Do not suppress exceptions from writer.write.
If there is a failure in the Hadoop backend while calling
writer.write, we should remember this original exception,
and try to call writer.close(), but if that fails as well,
still report the original exception.

Note that, if writer.write fails, it is likely that writer
was left in an invalid state, and so actually makes it more
likely that writer.close will also fail. Which just increases
the chances for writer.write's exception to be suppressed.

This patch introduces an admittedly potentially too cute
Utils.tryWithSafeFinally method to handle the try/finally
gyrations.

Author: Stephen Haberman <stephen@exigencecorp.com>

Closes #5223 from stephenh/do_not_suppress_writer_exception and squashes the following commits:

c7ad53f [Stephen Haberman] [SPARK-6560][CORE] Do not suppress exceptions from writer.write.
2015-04-03 09:48:37 +01:00
Reynold Xin 82701ee25f [SPARK-6428] Turn on explicit type checking for public methods.
This builds on my earlier pull requests and turns on the explicit type checking in scalastyle.

Author: Reynold Xin <rxin@databricks.com>

Closes #5342 from rxin/SPARK-6428 and squashes the following commits:

7b531ab [Reynold Xin] import ordering
2d9a8a5 [Reynold Xin] jl
e668b1c [Reynold Xin] override
9b9e119 [Reynold Xin] Parenthesis.
82e0cf5 [Reynold Xin] [SPARK-6428] Turn on explicit type checking for public methods.
2015-04-03 01:25:02 -07:00
zsxwing 440ea31b76 [SPARK-6621][Core] Fix the bug that calling EventLoop.stop in EventLoop.onReceive/onError/onStart doesn't call onStop
Author: zsxwing <zsxwing@gmail.com>

Closes #5280 from zsxwing/SPARK-6621 and squashes the following commits:

521125e [zsxwing] Fix the bug that calling EventLoop.stop in EventLoop.onReceive and EventLoop.onError doesn't call onStop
2015-04-02 22:54:30 -07:00
KaiXinXiaoLei 8a0aa81ca3 [CORE] The descriptionof jobHistory config should be spark.history.fs.logDirectory
The config option  is spark.history.fs.logDirectory, not spark.fs.history.logDirectory. So the descriptionof  should be changed. Thanks.

Author: KaiXinXiaoLei <huleilei1@huawei.com>

Closes #5332 from KaiXinXiaoLei/historyConfig and squashes the following commits:

5ffbfb5 [KaiXinXiaoLei] the describe of jobHistory config is error
2015-04-02 20:24:31 -07:00
Marcelo Vanzin 45134ec920 [SPARK-6650] [core] Stop ExecutorAllocationManager when context stops.
This fixes the thread leak. I also changed the unit test to keep track
of allocated contexts and make sure they're closed after tests are
run; this is needed since some tests use this pattern:

    val sc = createContext()
    doSomethingThatMayThrow()
    sc.stop()

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #5311 from vanzin/SPARK-6650 and squashes the following commits:

652c73b [Marcelo Vanzin] Nits.
5711512 [Marcelo Vanzin] More exception safety.
cc5a744 [Marcelo Vanzin] Stop alloc manager before scheduler.
9886f69 [Marcelo Vanzin] [SPARK-6650] [core] Stop ExecutorAllocationManager when context stops.
2015-04-02 19:48:55 -07:00
Hung Lin e3202aa2e9 SPARK-6414: Spark driver failed with NPE on job cancelation
Use Option for ActiveJob.properties to avoid NPE bug

Author: Hung Lin <hung.lin@gmail.com>

Closes #5124 from hunglin/SPARK-6414 and squashes the following commits:

2290b6b [Hung Lin] [SPARK-6414][core] Fix NPE in SparkContext.cancelJobGroup()
2015-04-02 14:01:43 -07:00
Davies Liu 0cce5451ad [SPARK-6667] [PySpark] remove setReuseAddress
The reused address on server side had caused the server can not acknowledge the connected connections, remove it.

This PR will retry once after timeout, it also add a timeout at client side.

Author: Davies Liu <davies@databricks.com>

Closes #5324 from davies/collect_hang and squashes the following commits:

e5a51a2 [Davies Liu] remove setReuseAddress
7977c2f [Davies Liu] do retry on client side
b838f35 [Davies Liu] retry after timeout
2015-04-02 12:18:33 -07:00
Patrick Wendell 6562787b96 [SPARK-6627] Some clean-up in shuffle code.
Before diving into review #4450 I did a look through the existing shuffle
code to learn how it works. Unfortunately, there are some very
confusing things in this code. This patch makes a few small changes
to simplify things. It is not easily to concisely describe the changes
because of how convoluted the issues were, but they are fairly small
logically:

1. There is a trait named `ShuffleBlockManager` that only deals with
   one logical function which is retrieving shuffle block data given shuffle
   block coordinates. This trait has two implementors FileShuffleBlockManager
   and IndexShuffleBlockManager. Confusingly the vast majority of those
   implementations have nothing to do with this particular functionality.
   So I've renamed the trait to ShuffleBlockResolver and documented it.
2. The aforementioned trait had two almost identical methods, for no good
   reason. I removed one method (getBytes) and modified callers to use the
   other one. I think the behavior is preserved in all cases.
3. The sort shuffle code uses an identifier "0" in the reduce slot of a
   BlockID as a placeholder. I made it into a constant since it needs to
   be consistent across multiple places.

I think for (3) there is actually a better solution that would avoid the
need to do this type of workaround/hack in the first place, but it's more
complex so I'm punting it for now.

Author: Patrick Wendell <patrick@databricks.com>

Closes #5286 from pwendell/cleanup and squashes the following commits:

c71fbc7 [Patrick Wendell] Open interface back up for testing
f36edd5 [Patrick Wendell] Code review feedback
d1c0494 [Patrick Wendell] Style fix
a406079 [Patrick Wendell] [HOTFIX] Some clean-up in shuffle code.
2015-04-01 23:42:09 -07:00
Kousuke Saruta d824c11c9f [SPARK-6597][Minor] Replace input:checkbox with input[type="checkbox"] in additional-metrics.js
In additional-metrics.js, there are some selector notation like `input:checkbox` but JQuery's official document says `input[type="checkbox"]` is better.

https://api.jquery.com/checkbox-selector/

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

Closes #5254 from sarutak/SPARK-6597 and squashes the following commits:

a253bc4 [Kousuke Saruta] Replaced input:checkbox with input[type="checkbox"]
2015-04-01 11:11:56 +01:00
Ilya Ganelin ff1915e12e [SPARK-4655][Core] Split Stage into ShuffleMapStage and ResultStage subclasses
Hi all - this patch changes the Stage class to an abstract class and introduces two new classes that extend it: ShuffleMapStage and ResultStage - with the goal of increasing readability of the DAGScheduler class. Their usage is updated within DAGScheduler.

Author: Ilya Ganelin <ilya.ganelin@capitalone.com>
Author: Ilya Ganelin <ilganeli@gmail.com>

Closes #4708 from ilganeli/SPARK-4655 and squashes the following commits:

c248924 [Ilya Ganelin] Merge branch 'SPARK-4655' of github.com:ilganeli/spark into SPARK-4655
d930385 [Ilya Ganelin] Fixed merge conflict from
a9a765f [Ilya Ganelin] Update DAGScheduler.scala
c03563c [Ilya Ganelin] Minor fixeS
c39e971 [Ilya Ganelin] Added return typing for public methods
845bc87 [Ilya Ganelin] Merge branch 'SPARK-4655' of github.com:ilganeli/spark into SPARK-4655
e8031d8 [Ilya Ganelin] Minor string fixes
4ec53ac [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-4655
c004f62 [Ilya Ganelin] Update DAGScheduler.scala
a2cb03f [Ilya Ganelin] [SPARK-4655] Replaced usages of Nil and eliminated some code reuse
3d5cf20 [Ilya Ganelin] [SPARK-4655] Moved mima exclude to 1.4
6912c55 [Ilya Ganelin] Resolved merge conflict
4bff208 [Ilya Ganelin] Minor stylistic fixes
c6fffbb [Ilya Ganelin] newline
41402ad [Ilya Ganelin] Style fixes
02c6981 [Ilya Ganelin] Merge branch 'SPARK-4655' of github.com:ilganeli/spark into SPARK-4655
c755a09 [Ilya Ganelin] Some more stylistic updates and minor refactoring
b6257a0 [Ilya Ganelin] Update MimaExcludes.scala
0f0c624 [Ilya Ganelin] Fixed merge conflict
2eba262 [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-4655
6b43d7b [Ilya Ganelin] Got rid of some spaces
6f1a5db [Ilya Ganelin] Revert "More minor formatting and refactoring"
1b3471b [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-4655
c9288e2 [Ilya Ganelin] More minor formatting and refactoring
d548caf [Ilya Ganelin] Formatting fix
c3ae5c2 [Ilya Ganelin] Explicit typing
0dacaf3 [Ilya Ganelin] Got rid of stale import
6da3a71 [Ilya Ganelin] Trailing whitespace
b85c5fe [Ilya Ganelin] Added minor fixes
a57dfcd [Ilya Ganelin] Added MiMA exclusion to get around binary compatibility check
83ed849 [Ilya Ganelin] moved braces for consistency
96dd161 [Ilya Ganelin] Fixed minor style error
cfd6f10 [Ilya Ganelin] Updated DAGScheduler to use new ResultStage and ShuffleMapStage classes
83494e9 [Ilya Ganelin] Added new Stage classes
2015-04-01 11:09:00 +01:00
Josh Rosen 37326079d8 [SPARK-6614] OutputCommitCoordinator should clear authorized committer only after authorized committer fails, not after any failure
In OutputCommitCoordinator, there is some logic to clear the authorized committer's lock on committing in case that task fails.  However, it looks like the current code also clears this lock if other non-authorized tasks fail, which is an obvious bug.

In theory, it's possible that this could allow a new committer to start, run to completion, and commit output before the authorized committer finished, but it's unlikely that this race occurs often in practice due to the complex combination of failure and timing conditions that would be required to expose it.

This patch addresses this issue and adds a regression test.

Thanks to aarondav for spotting this issue.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #5276 from JoshRosen/SPARK-6614 and squashes the following commits:

d532ba7 [Josh Rosen] Check whether failed task was authorized committer
cbb3784 [Josh Rosen] Add regression test for SPARK-6614
2015-03-31 16:18:39 -07:00
zsxwing 56775571cb [SPARK-5124][Core] Move StopCoordinator to the receive method since it does not require a reply
Hotfix for #4588

cc rxin

Author: zsxwing <zsxwing@gmail.com>

Closes #5283 from zsxwing/hotfix and squashes the following commits:

cf3e5a7 [zsxwing] Move StopCoordinator to the receive method since it does not require a reply
2015-03-30 22:10:49 -07:00
Cheng Lian fde6945417 [SPARK-6369] [SQL] Uses commit coordinator to help committing Hive and Parquet tables
This PR leverages the output commit coordinator introduced in #4066 to help committing Hive and Parquet tables.

This PR extracts output commit code in `SparkHadoopWriter.commit` to `SparkHadoopMapRedUtil.commitTask`, and reuses it for committing Parquet and Hive tables on executor side.

TODO

- [ ] Add tests

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

Author: Cheng Lian <lian@databricks.com>

Closes #5139 from liancheng/spark-6369 and squashes the following commits:

72eb628 [Cheng Lian] Fixes typo in javadoc
9a4b82b [Cheng Lian] Adds javadoc and addresses @aarondav's comments
dfdf3ef [Cheng Lian] Uses commit coordinator to help committing Hive and Parquet tables
2015-03-31 07:48:37 +08:00
Li Zhihui 01dc9f50d1 Fix string interpolator error in HeartbeatReceiver
Error log before fixed
<code>15/03/29 10:07:25 ERROR YarnScheduler: Lost an executor 24 (already removed): Executor heartbeat timed out after ${now - lastSeenMs} ms</code>

Author: Li Zhihui <zhihui.li@intel.com>

Closes #5255 from li-zhihui/fixstringinterpolator and squashes the following commits:

c93f2b7 [Li Zhihui] Fix string interpolator error in HeartbeatReceiver
2015-03-29 21:30:37 -07:00
zsxwing a8d53afb4e [SPARK-5124][Core] A standard RPC interface and an Akka implementation
This PR added a standard internal RPC interface for Spark and an Akka implementation. See [the design document](https://issues.apache.org/jira/secure/attachment/12698710/Pluggable%20RPC%20-%20draft%202.pdf) for more details.

I will split the whole work into multiple PRs to make it easier for code review. This is the first PR and avoid to touch too many files.

Author: zsxwing <zsxwing@gmail.com>

Closes #4588 from zsxwing/rpc-part1 and squashes the following commits:

fe3df4c [zsxwing] Move registerEndpoint and use actorSystem.dispatcher in asyncSetupEndpointRefByURI
f6f3287 [zsxwing] Remove RpcEndpointRef.toURI
8bd1097 [zsxwing] Fix docs and the code style
f459380 [zsxwing] Add RpcAddress.fromURI and rename urls to uris
b221398 [zsxwing] Move send methods above ask methods
15cfd7b [zsxwing] Merge branch 'master' into rpc-part1
9ffa997 [zsxwing] Fix MiMa tests
78a1733 [zsxwing] Merge remote-tracking branch 'origin/master' into rpc-part1
385b9c3 [zsxwing] Fix the code style and add docs
2cc3f78 [zsxwing] Add an asynchronous version of setupEndpointRefByUrl
e8dfec3 [zsxwing] Remove 'sendWithReply(message: Any, sender: RpcEndpointRef): Unit'
08564ae [zsxwing] Add RpcEnvFactory to create RpcEnv
e5df4ca [zsxwing] Handle AkkaFailure(e) in Actor
ec7c5b0 [zsxwing] Fix docs
7fc95e1 [zsxwing] Implement askWithReply in RpcEndpointRef
9288406 [zsxwing] Document thread-safety for setupThreadSafeEndpoint
3007c09 [zsxwing] Move setupDriverEndpointRef to RpcUtils and rename to makeDriverRef
c425022 [zsxwing] Fix the code style
5f87700 [zsxwing] Move the logical of processing message to a private function
3e56123 [zsxwing] Use lazy to eliminate CountDownLatch
07f128f [zsxwing] Remove ActionScheduler.scala
4d34191 [zsxwing] Remove scheduler from RpcEnv
7cdd95e [zsxwing] Add docs for RpcEnv
51e6667 [zsxwing] Add 'sender' to RpcCallContext and rename the parameter of receiveAndReply to 'context'
ffc1280 [zsxwing] Rename 'fail' to 'sendFailure' and other minor code style changes
28e6d0f [zsxwing] Add onXXX for network events and remove the companion objects of network events
3751c97 [zsxwing] Rename RpcResponse to RpcCallContext
fe7d1ff [zsxwing] Add explicit reply in rpc
7b9e0c9 [zsxwing] Fix the indentation
04a106e [zsxwing] Remove NopCancellable and add a const NOP in object SettableCancellable
2a579f4 [zsxwing] Remove RpcEnv.systemName
155b987 [zsxwing] Change newURI to uriOf and add some comments
45b2317 [zsxwing] A standard RPC interface and An Akka implementation
2015-03-29 21:25:09 -07:00
June.He 0e2753ff14 [SPARK-6585][Tests]Fix FileServerSuite testcase in some Env.
Change FileServerSuite.test("HttpFileServer should not work with SSL when the server is untrusted") catch SSLException

Author: June.He <jun.hejun@huawei.com>

Closes #5239 from sisihj/SPARK-6585 and squashes the following commits:

cb19ae3 [June.He] Change FileServerSuite.test("HttpFileServer should not work with SSL when the server is untrusted") catch SSLException
2015-03-29 12:47:22 +01:00
Thomas Graves 52ece26b8f [SPARK-6558] Utils.getCurrentUserName returns the full principal name instead of login name
Utils.getCurrentUserName returns UserGroupInformation.getCurrentUser().getUserName() when SPARK_USER isn't set. It should return UserGroupInformation.getCurrentUser().getShortUserName()
getUserName() returns the users full principal name (ie user1CORP.COM). getShortUserName() returns just the users login name (user1).

This just happens to work on YARN because the Client code sets:
env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName()

Author: Thomas Graves <tgraves@apache.org>

Closes #5229 from tgravescs/SPARK-6558 and squashes the following commits:

24830bf [Thomas Graves] Utils.getCurrentUserName returns the full principal name instead of login name
2015-03-29 12:43:30 +01:00
zsxwing da546b7ba0 [SPARK-6556][Core] Fix wrong parsing logic of executorTimeoutMs and checkTimeoutIntervalMs in HeartbeatReceiver
The current reading logic of `executorTimeoutMs` is:
```Scala
private val executorTimeoutMs = sc.conf.getLong("spark.network.timeout",
    sc.conf.getLong("spark.storage.blockManagerSlaveTimeoutMs", 120)) * 1000
```
So if `spark.storage.blockManagerSlaveTimeoutMs` is 10000 and `spark.network.timeout` is not set, executorTimeoutMs will be 10000 * 1000. But the correct value should have been 10000.

`checkTimeoutIntervalMs` has the same issue.

This PR fixes them.

Author: zsxwing <zsxwing@gmail.com>

Closes #5209 from zsxwing/SPARK-6556 and squashes the following commits:

6a0a411 [zsxwing] Fix docs
c7d5422 [zsxwing] Add comments for executorTimeoutMs and checkTimeoutIntervalMs
ccd5147 [zsxwing] Fix wrong parsing logic of executorTimeoutMs and checkTimeoutIntervalMs in HeartbeatReceiver
2015-03-27 12:31:06 +00:00
mcheah 49d2ec63ec [SPARK-6405] Limiting the maximum Kryo buffer size to be 2GB.
Kryo buffers are backed by byte arrays, but primitive arrays can only be
up to 2GB in size. It is misleading to allow users to set buffers past
this size.

Author: mcheah <mcheah@palantir.com>

Closes #5218 from mccheah/feature/limit-kryo-buffer and squashes the following commits:

1d6d1be [mcheah] Fixing numeric typo
e2e30ce [mcheah] Removing explicit int and double type to match style
09fd80b [mcheah] Should be >= not >. Slightly more consistent error message.
60634f9 [mcheah] [SPARK-6405] Limiting the maximum Kryo buffer size to be 2GB.
2015-03-26 22:48:42 -07:00
Sean Owen fe15ea9760 SPARK-6480 [CORE] histogram() bucket function is wrong in some simple edge cases
Fix fastBucketFunction for histogram() to handle edge conditions more correctly. Add a test, and fix existing one accordingly

Author: Sean Owen <sowen@cloudera.com>

Closes #5148 from srowen/SPARK-6480 and squashes the following commits:

974a0a0 [Sean Owen] Additional test of huge ranges, and a few more comments (and comment fixes)
23ec01e [Sean Owen] Fix fastBucketFunction for histogram() to handle edge conditions more correctly. Add a test, and fix existing one accordingly
2015-03-26 15:00:23 +00:00
zsxwing 0c88ce5416 [SPARK-6468][Block Manager] Fix the race condition of subDirs in DiskBlockManager
There are two race conditions of `subDirs` in `DiskBlockManager`:

1. `getAllFiles` does not use correct locks to read the contents in `subDirs`. Although it's designed for testing, it's still worth to add correct locks to eliminate the race condition.
2. The double-check has a race condition in `getFile(filename: String)`. If a thread finds `subDirs(dirId)(subDirId)` is not null out of the `synchronized` block, it may not be able to see the correct content of the File instance pointed by `subDirs(dirId)(subDirId)` according to the Java memory model (there is no volatile variable here).

This PR fixed the above race conditions.

Author: zsxwing <zsxwing@gmail.com>

Closes #5136 from zsxwing/SPARK-6468 and squashes the following commits:

cbb872b [zsxwing] Fix the race condition of subDirs in DiskBlockManager
2015-03-26 12:54:48 +00:00
Josh Rosen d44a3362ed [SPARK-6079] Use index to speed up StatusTracker.getJobIdsForGroup()
`StatusTracker.getJobIdsForGroup()` is implemented via a linear scan over a HashMap rather than using an index, which might be an expensive operation if there are many (e.g. thousands) of retained jobs.

This patch adds a new map to `JobProgressListener` in order to speed up these lookups.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #4830 from JoshRosen/statustracker-job-group-indexing and squashes the following commits:

e39c5c7 [Josh Rosen] Address review feedback
6709fb2 [Josh Rosen] Merge remote-tracking branch 'origin/master' into statustracker-job-group-indexing
2c49614 [Josh Rosen] getOrElse
97275a7 [Josh Rosen] Add jobGroup to jobId index to JobProgressListener
2015-03-25 17:40:00 -07:00
Andrew Or c1b74df604 [SPARK-5771] Master UI inconsistently displays application cores
If the user calls `sc.stop()`, then the number of cores under "Completed Applications" will be 0. If the user does not call `sc.stop()`, then the number of cores will be however many cores were being used before the application exited. This PR makes both cases have the behavior of the latter.

Note that there have been a series of PR that attempted to fix this. For the full discussion, please refer to #4841. The unregister event is necessary because of a subtle race condition explained in that PR.

Tested this locally with and without calling `sc.stop()`.

Author: Andrew Or <andrew@databricks.com>

Closes #5177 from andrewor14/master-ui-cores and squashes the following commits:

62449d1 [Andrew Or] Freeze application state before finishing it
2015-03-25 13:28:32 -07:00
Kousuke Saruta acef51defb [SPARK-6537] UIWorkloadGenerator: The main thread should not stop SparkContext until all jobs finish
The main thread of UIWorkloadGenerator spawn sub threads to launch jobs but the main thread stop SparkContext without waiting for finishing those threads.

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

Closes #5187 from sarutak/SPARK-6537 and squashes the following commits:

4e9307a [Kousuke Saruta] Fixed UIWorkloadGenerator so that the main thread stop SparkContext after all jobs finish
2015-03-25 13:27:15 -07:00
zsxwing 883b7e9030 [SPARK-6076][Block Manager] Fix a potential OOM issue when StorageLevel is MEMORY_AND_DISK_SER
In dcd1e42d6b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala (L538) , when StorageLevel is `MEMORY_AND_DISK_SER`, it will copy the content from file into memory, then put it into MemoryStore.
```scala
              val copyForMemory = ByteBuffer.allocate(bytes.limit)
              copyForMemory.put(bytes)
              memoryStore.putBytes(blockId, copyForMemory, level)
              bytes.rewind()
```
However, if the file is bigger than the free memory, OOM will happen. A better approach is testing if there is enough memory. If not, copyForMemory should not be created, since this is an optional operation.

Author: zsxwing <zsxwing@gmail.com>

Closes #4827 from zsxwing/SPARK-6076 and squashes the following commits:

7d25545 [zsxwing] Add alias for tryToPut and dropFromMemory
1100a54 [zsxwing] Replace call-by-name with () => T
0cc0257 [zsxwing] Fix a potential OOM issue when StorageLevel is MEMORY_AND_DISK_SER
2015-03-25 12:17:18 -07:00
Xiangrui Meng c14ddd97ed [SPARK-6515] update OpenHashSet impl
Though I don't see any bug in the existing code, the update in this PR makes it read better. rxin

Author: Xiangrui Meng <meng@databricks.com>

Closes #5176 from mengxr/SPARK-6515 and squashes the following commits:

134494d [Xiangrui Meng] update OpenHashSet impl
2015-03-24 18:58:27 -07:00
Xiangrui Meng 6930e965e2 [SPARK-6512] add contains to OpenHashMap
Add `contains` to test whether a key exists in an OpenHashMap. rxin

Author: Xiangrui Meng <meng@databricks.com>

Closes #5171 from mengxr/openhashmap-contains and squashes the following commits:

d6e6f1f [Xiangrui Meng] add contains to primitivekeyopenhashmap
748a69b [Xiangrui Meng] add contains to OpenHashMap
2015-03-24 17:06:22 -07:00
Andrew Or dd907d1a9d Revert "[SPARK-5771] Number of Cores in Completed Applications of Standalone Master Web Page always be 0 if sc.stop() is called"
This reverts commit dd077abf2e.

Conflicts:
	core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala
	core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
2015-03-24 16:49:27 -07:00
Andrew Or f7c3668ee6 Revert "[SPARK-5771][UI][hotfix] Change Requested Cores into * if default cores is not set"
This reverts commit 12135e9054.
2015-03-24 16:41:31 -07:00
Kay Ousterhout d8ccf655f3 [SPARK-3570] Include time to open files in shuffle write time.
Opening shuffle files can be very significant when the disk is
contended, especially when using ext3. While writing data to
a file can avoid hitting disk (and instead hit the buffer
cache), opening a file always involves writing some metadata
about the file to disk, so the open time can be a very significant
portion of the shuffle write time. In one job I ran recently, the time to
write shuffle data to the file was only 4ms for each task, but
the time to open the file was about 100x as long (~400ms).

When we add metrics about spilled data (#2504), we should ensure
that the file open time is also included there.

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #4550 from kayousterhout/SPARK-3570 and squashes the following commits:

ea3a4ae [Kay Ousterhout] Added comment about excluded open time
fdc5185 [Kay Ousterhout] Improved comment
42b7e43 [Kay Ousterhout] Fixed parens for nanotime
2423555 [Kay Ousterhout] [SPARK-3570] Include time to open files in shuffle write time.
2015-03-24 16:29:40 -07:00
Kay Ousterhout 6948ab6f8b [SPARK-6088] Correct how tasks that get remote results are shown in UI.
It would be great to fix this for 1.3. since the fix is surgical and it helps understandability for users.

cc shivaram pwendell

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #4839 from kayousterhout/SPARK-6088 and squashes the following commits:

3ab012c [Kay Ousterhout] Update getting result time incrementally, correctly set GET_RESULT status
f346b49 [Kay Ousterhout] Typos
748ea6b [Kay Ousterhout] Fixed build failure
84d617c [Kay Ousterhout] [SPARK-6088] Correct how tasks that get remote results are shown in the UI.
2015-03-24 16:26:43 -07:00
Kousuke Saruta 85cf063682 [SPARK-5559] [Streaming] [Test] Remove oppotunity we met flakiness when running FlumeStreamSuite
When we run FlumeStreamSuite on Jenkins, sometimes we get error like as follows.

    sbt.ForkMain$ForkError: The code passed to eventually never returned normally. Attempted 52 times over 10.094849836 seconds. Last failure message: Error connecting to localhost/127.0.0.1:23456.
	    at org.scalatest.concurrent.Eventually$class.tryTryAgain$1(Eventually.scala:420)
	    at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:438)
	    at org.scalatest.concurrent.Eventually$.eventually(Eventually.scala:478)
	    at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:307)
	   at org.scalatest.concurrent.Eventually$.eventually(Eventually.scala:478)
	   at org.apache.spark.streaming.flume.FlumeStreamSuite.writeAndVerify(FlumeStreamSuite.scala:116)
           at org.apache.spark.streaming.flume.FlumeStreamSuite.org$apache$spark$streaming$flume$FlumeStreamSuite$$testFlumeStream(FlumeStreamSuite.scala:74)
	   at org.apache.spark.streaming.flume.FlumeStreamSuite$$anonfun$3.apply$mcV$sp(FlumeStreamSuite.scala:66)
	    at org.apache.spark.streaming.flume.FlumeStreamSuite$$anonfun$3.apply(FlumeStreamSuite.scala:66)
	    at org.apache.spark.streaming.flume.FlumeStreamSuite$$anonfun$3.apply(FlumeStreamSuite.scala:66)
	    at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
	    at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
	    at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
	    at org.scalatest.Transformer.apply(Transformer.scala:22)
	    at org.scalatest.Transformer.apply(Transformer.scala:20)
    	    at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166)
	    at org.scalatest.Suite$class.withFixture(Suite.scala:1122)
	    at org.scalatest.FunSuite.withFixture(FunSuite.scala:1555)
	    at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163)
	   at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
	    at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
	    at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
	    at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175)

This error is caused by check-then-act logic  when it find free-port .

      /** Find a free port */
      private def findFreePort(): Int = {
        Utils.startServiceOnPort(23456, (trialPort: Int) => {
          val socket = new ServerSocket(trialPort)
          socket.close()
          (null, trialPort)
        }, conf)._2
      }

Removing the check-then-act is not easy but we can reduce the chance of having the error by choosing random value for initial port instead of 23456.

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

Closes #4337 from sarutak/SPARK-5559 and squashes the following commits:

16f109f [Kousuke Saruta] Added `require` to Utils#startServiceOnPort
c39d8b6 [Kousuke Saruta] Merge branch 'SPARK-5559' of github.com:sarutak/spark into SPARK-5559
1610ba2 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-5559
33357e3 [Kousuke Saruta] Changed "findFreePort" method in MQTTStreamSuite and FlumeStreamSuite so that it can choose valid random port
a9029fe [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-5559
9489ef9 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-5559
8212e42 [Kousuke Saruta] Modified default port used in FlumeStreamSuite from 23456 to random value
2015-03-24 16:20:52 +00:00
Reynold Xin 4ce2782a61 [SPARK-6428] Added explicit types for all public methods in core.
Author: Reynold Xin <rxin@databricks.com>

Closes #5125 from rxin/core-explicit-type and squashes the following commits:

f471415 [Reynold Xin] Revert style checker changes.
81b66e4 [Reynold Xin] Code review feedback.
a7533e3 [Reynold Xin] Mima excludes.
1d795f5 [Reynold Xin] [SPARK-6428] Added explicit types for all public methods in core.
2015-03-23 23:41:06 -07:00
Patrick Wendell 6cd7058b36 Revert "[SPARK-6122][Core] Upgrade Tachyon client version to 0.6.1."
This reverts commit a41b9c6004.
2015-03-23 15:08:39 -07:00
Calvin Jia a41b9c6004 [SPARK-6122][Core] Upgrade Tachyon client version to 0.6.1.
Changes the Tachyon client version from 0.5 to 0.6 in spark core and distribution script.

New dependencies in Tachyon 0.6.0 include

commons-codec:commons-codec:jar:1.5:compile
io.netty:netty-all:jar:4.0.23.Final:compile

These are already in spark core.

Author: Calvin Jia <jia.calvin@gmail.com>

Closes #4867 from calvinjia/upgrade_tachyon_0.6.0 and squashes the following commits:

eed9230 [Calvin Jia] Update tachyon version to 0.6.1.
11907b3 [Calvin Jia] Use TachyonURI for tachyon paths instead of strings.
71bf441 [Calvin Jia] Upgrade Tachyon client version to 0.6.0.
2015-03-22 11:11:29 -07:00
Jongyoul Lee adb2ff752f [SPARK-6453][Mesos] Some Mesos*Suite have a different package with their classes
- Moved Suites from o.a.s.s.mesos to o.a.s.s.cluster.mesos

Author: Jongyoul Lee <jongyoul@gmail.com>

Closes #5126 from jongyoul/SPARK-6453 and squashes the following commits:

4f24a3e [Jongyoul Lee] [SPARK-6453][Mesos] Some Mesos*Suite have a different package with their classes - Fixed imports orders
8ab149d [Jongyoul Lee] [SPARK-6453][Mesos] Some Mesos*Suite have a different package with their classes - Moved Suites from o.a.s.s.mesos to o.a.s.s.cluster.mesos
2015-03-22 15:54:19 +00:00
Ryan Williams b9fe504b49 [SPARK-6448] Make history server log parse exceptions
This helped me to debug a parse error that was due to the event log format changing recently.

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

Closes #5122 from ryan-williams/histerror and squashes the following commits:

5831656 [Ryan Williams] line length
c3742ae [Ryan Williams] Make history server log parse exceptions
2015-03-22 11:54:23 +00:00
Jongyoul Lee 49a01c7ea2 [SPARK-6423][Mesos] MemoryUtils should use memoryOverhead if it's set
- Fixed calculateTotalMemory to use spark.mesos.executor.memoryOverhead
- Added testCase

Author: Jongyoul Lee <jongyoul@gmail.com>

Closes #5099 from jongyoul/SPARK-6423 and squashes the following commits:

6747fce [Jongyoul Lee] [SPARK-6423][Mesos] MemoryUtils should use memoryOverhead if it's set - Changed a description of spark.mesos.executor.memoryOverhead
475a7c8 [Jongyoul Lee] [SPARK-6423][Mesos] MemoryUtils should use memoryOverhead if it's set - Fit the import rules
453c5a2 [Jongyoul Lee] [SPARK-6423][Mesos] MemoryUtils should use memoryOverhead if it's set - Fixed calculateTotalMemory to use spark.mesos.executor.memoryOverhead - Added testCase
2015-03-20 19:14:35 +00:00
Marcelo Vanzin a74564591f [SPARK-6371] [build] Update version to 1.4.0-SNAPSHOT.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #5056 from vanzin/SPARK-6371 and squashes the following commits:

63220df [Marcelo Vanzin] Merge branch 'master' into SPARK-6371
6506f75 [Marcelo Vanzin] Use more fine-grained exclusion.
178ba71 [Marcelo Vanzin] Oops.
75b2375 [Marcelo Vanzin] Exclude VertexRDD in MiMA.
a45a62c [Marcelo Vanzin] Work around MIMA warning.
1d8a670 [Marcelo Vanzin] Re-group jetty exclusion.
0e8e909 [Marcelo Vanzin] Ignore ml, don't ignore graphx.
cef4603 [Marcelo Vanzin] Indentation.
296cf82 [Marcelo Vanzin] [SPARK-6371] [build] Update version to 1.4.0-SNAPSHOT.
2015-03-20 18:43:57 +00:00
mbonaci 28bcb9e9e8 [SPARK-6370][core] Documentation: Improve all 3 docs for RDD.sample
The docs for the `sample` method were insufficient, now less so.

Author: mbonaci <mbonaci@gmail.com>

Closes #5097 from mbonaci/master and squashes the following commits:

a6a9d97 [mbonaci] [SPARK-6370][core] Documentation: Improve all 3 docs for RDD.sample method
2015-03-20 18:33:53 +00:00
Sean Owen 6f80c3e888 SPARK-6338 [CORE] Use standard temp dir mechanisms in tests to avoid orphaned temp files
Use `Utils.createTempDir()` to replace other temp file mechanisms used in some tests, to further ensure they are cleaned up, and simplify

Author: Sean Owen <sowen@cloudera.com>

Closes #5029 from srowen/SPARK-6338 and squashes the following commits:

27b740a [Sean Owen] Fix hive-thriftserver tests that don't expect an existing dir
4a212fa [Sean Owen] Standardize a bit more temp dir management
9004081 [Sean Owen] Revert some added recursive-delete calls
57609e4 [Sean Owen] Use Utils.createTempDir() to replace other temp file mechanisms used in some tests, to further ensure they are cleaned up, and simplify
2015-03-20 14:16:21 +00:00
Jongyoul Lee 116c553fd6 [SPARK-6286][Mesos][minor] Handle missing Mesos case TASK_ERROR
- Made TaskState.isFailed for handling TASK_LOST and TASK_ERROR and synchronizing CoarseMesosSchedulerBackend and MesosSchedulerBackend
- This is related #5000

Author: Jongyoul Lee <jongyoul@gmail.com>

Closes #5088 from jongyoul/SPARK-6286-1 and squashes the following commits:

4f2362f [Jongyoul Lee] [SPARK-6286][Mesos][minor] Handle missing Mesos case TASK_ERROR - Fixed scalastyle
ac4336a [Jongyoul Lee] [SPARK-6286][Mesos][minor] Handle missing Mesos case TASK_ERROR - Made TaskState.isFailed for handling TASK_LOST and TASK_ERROR and synchronizing CoarseMesosSchedulerBackend and MesosSchedulerBackend
2015-03-20 12:24:34 +00:00
Reynold Xin 0745a305fa Tighten up field/method visibility in Executor and made some code more clear to read.
I was reading Executor just now and found that some latest changes introduced some weird code path with too much monadic chaining and unnecessary fields. I cleaned it up a bit, and also tightened up the visibility of various fields/methods. Also added some inline documentation to help understand this code better.

Author: Reynold Xin <rxin@databricks.com>

Closes #4850 from rxin/executor and squashes the following commits:

866fc60 [Reynold Xin] Code review feedback.
020efbb [Reynold Xin] Tighten up field/method visibility in Executor and made some code more clear to read.
2015-03-19 22:12:01 -04:00
Wenchen Fan 3b5aaa6a5f [Core][minor] remove unused visitedStages in DAGScheduler.stageDependsOn
We define and update `visitedStages` in `DAGScheduler.stageDependsOn`, but never read it. So we can safely remove it.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #5086 from cloud-fan/minor and squashes the following commits:

24663ea [Wenchen Fan] remove un-used variable
2015-03-19 15:25:32 -04:00
mcheah 3c4e486b9c [SPARK-5843] [API] Allowing map-side combine to be specified in Java.
Specifically, when calling JavaPairRDD.combineByKey(), there is a new
six-parameter method that exposes the map-side-combine boolean as the
fifth parameter and the serializer as the sixth parameter.

Author: mcheah <mcheah@palantir.com>

Closes #4634 from mccheah/pair-rdd-map-side-combine and squashes the following commits:

5c58319 [mcheah] Fixing compiler errors.
3ce7deb [mcheah] Addressing style and documentation comments.
7455c7a [mcheah] Allowing Java combineByKey to specify Serializer as well.
6ddd729 [mcheah] [SPARK-5843] Allowing map-side combine to be specified in Java.
2015-03-19 08:51:49 -04:00
CodingCat 2c3f83c34b [SPARK-4012] stop SparkContext when the exception is thrown from an infinite loop
https://issues.apache.org/jira/browse/SPARK-4012

This patch is a resubmission for https://github.com/apache/spark/pull/2864

What I am proposing in this patch is that ***when the exception is thrown from an infinite loop, we should stop the SparkContext, instead of let JVM throws exception forever***

So, in the infinite loops where we originally wrapped with a ` logUncaughtExceptions`, I changed to `tryOrStopSparkContext`, so that the Spark component is stopped

Early stopped JVM process is helpful for HA scheme design, for example,

The user has a script checking the existence of the pid of the Spark Streaming driver for monitoring the availability; with the code before this patch, the JVM process is still available but not functional when the exceptions are thrown

andrewor14, srowen , mind taking further consideration about the change?

Author: CodingCat <zhunansjtu@gmail.com>

Closes #5004 from CodingCat/SPARK-4012-1 and squashes the following commits:

589276a [CodingCat] throw fatal error again
3c72cd8 [CodingCat] address the comments
6087864 [CodingCat] revise comments
6ad3eb0 [CodingCat] stop SparkContext instead of quit the JVM process
6322959 [CodingCat] exit JVM process when the exception is thrown from an infinite loop
2015-03-18 23:48:45 -07:00
Wenchen Fan 540b2a4eab [SPARK-6394][Core] cleanup BlockManager companion object and improve the getCacheLocs method in DAGScheduler
The current implementation include searching a HashMap many times, we can avoid this.
Actually if you look into `BlockManager.blockIdsToBlockManagers`, the core function call is [this](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/storage/BlockManager.scala#L1258), so we can call `blockManagerMaster.getLocations` directly and avoid building a HashMap.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #5043 from cloud-fan/small and squashes the following commits:

e959d12 [Wenchen Fan] fix style
203c493 [Wenchen Fan] some cleanup in BlockManager companion object
d409099 [Wenchen Fan] address rxin's comment
faec999 [Wenchen Fan] add regression test
2fb57aa [Wenchen Fan] imporve the getCacheLocs method
2015-03-18 19:43:04 -07:00
Marcelo Vanzin 981fbafa2a [SPARK-6325] [core,yarn] Do not change target executor count when killing executors.
The dynamic execution code has two ways to reduce the number of executors: one
where it reduces the total number of executors it wants, by asking for an absolute
number of executors that is lower than the previous one. The second is by
explicitly killing idle executors.

YarnAllocator was mixing those up and lowering the target number of executors
when a kill was issued. Instead, trust the frontend knows what it's doing, and kill
executors without messing with other accounting. That means that if the frontend
kills an executor without lowering the target, it will get a new executor shortly.

The one situation where both actions (lower the target and kill executor) need to
happen together is when user code explicitly calls `SparkContext.killExecutors`.
In that case, issue two calls to the backend to achieve the goal.

I also did some minor cleanup in related code:
- avoid sending a request for executors when target is unchanged, to avoid log
  spam in the AM
- avoid printing misleading log messages in the AM when there are no requests
  to cancel
- fix a slow memory leak plus misleading error message on the driver caused by
  failing to completely unregister the executor.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #5018 from vanzin/SPARK-6325 and squashes the following commits:

2e782a3 [Marcelo Vanzin] Avoid redundant logging on the AM side.
a3567cd [Marcelo Vanzin] Add parentheses.
a363926 [Marcelo Vanzin] Update logic.
a158101 [Marcelo Vanzin] [SPARK-6325] [core,yarn] Disallow reducing executor count past running count.
2015-03-18 09:18:28 -04:00
Iulian Dragos 9d112a958e [SPARK-6286][minor] Handle missing Mesos case TASK_ERROR.
Author: Iulian Dragos <jaguarul@gmail.com>

Closes #5000 from dragos/issue/task-error-case and squashes the following commits:

e063627 [Iulian Dragos] Handle TASK_ERROR in Mesos scheduler backends.
ac17cf0 [Iulian Dragos] Handle missing Mesos case TASK_ERROR.
2015-03-18 09:15:33 -04:00
nemccarthy 4cca3917dc [SPARK-6313] Add config option to disable file locks/fetchFile cache to ...
...support NFS mounts.

This is a work around for now with the goal to find a more permanent solution.
https://issues.apache.org/jira/browse/SPARK-6313

Author: nemccarthy <nathan@nemccarthy.me>

Closes #5036 from nemccarthy/master and squashes the following commits:

2eaaf42 [nemccarthy] [SPARK-6313] Update config wording doc for spark.files.useFetchCache
5de7eb4 [nemccarthy] [SPARK-6313] Add config option to disable file locks/fetchFile cache to support NFS mounts
2015-03-17 09:33:11 -07:00
Josh Rosen 0f673c21f6 [SPARK-3266] Use intermediate abstract classes to fix type erasure issues in Java APIs
This PR addresses a Scala compiler bug ([SI-8905](https://issues.scala-lang.org/browse/SI-8905)) that was breaking some of the Spark Java APIs.  In a nutshell, it seems that methods whose implementations are inherited from generic traits sometimes have their type parameters erased to Object.  This was causing methods like `DoubleRDD.min()` to throw confusing NoSuchMethodErrors at runtime.

The fix implemented here is to introduce an intermediate layer of abstract classes and inherit from those instead of directly extends the `Java*Like` traits.  This should not break binary compatibility.

I also improved the test coverage of the Java API, adding several new tests for methods that failed at runtime due to this bug.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #5050 from JoshRosen/javardd-si-8905-fix and squashes the following commits:

2feb068 [Josh Rosen] Use intermediate abstract classes to work around SPARK-3266
d5f3e5d [Josh Rosen] Add failing regression tests for SPARK-3266
2015-03-17 09:18:57 -07:00
Imran Rashid e9f22c6129 [SPARK-6365] jetty-security also needed for SPARK_PREPEND_CLASSES to work
https://issues.apache.org/jira/browse/SPARK-6365

thanks vanzin for helping me figure this out

Author: Imran Rashid <irashid@cloudera.com>

Closes #5052 from squito/fix_prepend_classes and squashes the following commits:

09d334c [Imran Rashid] jetty-security also needed for SPARK_PREPEND_CLASSES to work
2015-03-17 09:41:06 -05:00
mcheah 005d1c5f29 [SPARK-6269] [CORE] Use ScalaRunTime's array methods instead of java.lang.reflect.Array in size estimation
This patch switches the usage of java.lang.reflect.Array in Size estimation to using scala's RunTime array-getter methods. The notes on https://bugs.openjdk.java.net/browse/JDK-8051447 tipped me off to the fact that using java.lang.reflect.Array was not ideal. At first, I used the code from that ticket, but it turns out that ScalaRunTime's array-related methods avoid the bottleneck of invoking native code anyways, so that was sufficient to boost performance in size estimation.

The idea is to use pure Java code in implementing the methods there, as opposed to relying on native C code which ends up being ill-performing. This improves the performance of estimating the size of arrays when we are checking for spilling in Spark.

Here's the benchmark discussion from the ticket:

I did two tests. The first, less convincing, take-with-a-block-of-salt test I did was do a simple groupByKey operation to collect objects in a 4.0 GB text file RDD into 30,000 buckets. I ran 1 Master and 4 Spark Worker JVMs on my mac, fetching the RDD from a text file simply stored on disk, and saving it out to another file located on local disk. The wall clock times I got back before and after the change were:

Before: 352.195s, 343.871s, 359.080s
After (using code directly from the JDK ticket, not the scala code in this PR): 342.929583s, 329.456623s, 326.151481s

So, there is a bit of an improvement after the change. I also did some YourKit profiling of the executors to get an idea of how much time was spent in size estimation before and after the change. I roughly saw that size estimation took up less of the time after my change, but YourKit's profiling can be inconsistent and who knows if I was profiling the executors that had the same data between runs?

The more convincing test I did was to run the size-estimation logic itself in an isolated unit test. I ran the following code:
```
val bigArray = Array.fill(1000)(Array.fill(1000)(java.util.UUID.randomUUID().toString()))
test("String arrays only perf testing") {
  val startTime = System.currentTimeMillis()
  for (i <- 1 to 50000) {
    SizeEstimator.estimate(bigArray)
  }
  println("Runtime: " + (System.currentTimeMillis() - startTime) / 1000.0000)
}
```
I wanted to use a 2D array specifically because I wanted to measure the performance of repeatedly calling Array.getLength. I used UUID-Strings to ensure that the strings were randomized (so String object re-use doesn't happen), but that they would all be the same size. The results were as follows:

Before PR: 222.681 s, 218.34 s, 211.739s
After latest change: 170.715 s, 176.775 s, 180.298 s
.

Author: mcheah <mcheah@palantir.com>
Author: Justin Uang <justin.uang@gmail.com>

Closes #4972 from mccheah/feature/spark-6269-reflect-array and squashes the following commits:

8527852 [mcheah] Respect CamelCase for numElementsDrawn
18d4b50 [mcheah] Addressing style comments - while loops instead of for loops
16ce534 [mcheah] Organizing imports properly
db890ea [mcheah] Removing CastedArray and just using ScalaRunTime.
cb67ce2 [mcheah] Fixing a scalastyle error - line too long
5d53c4c [mcheah] Removing unused parameter in visitArray.
6467759 [mcheah] Including primitive size information inside CastedArray.
93f4b05 [mcheah] Using Scala instead of Java for the array-reflection implementation.
a557ab8 [mcheah] Using a wrapper around arrays to do casting only once
ca063fc [mcheah] Fixing a compiler error made while refactoring style
1fe09de [Justin Uang] [SPARK-6269] Use a different implementation of java.lang.reflect.Array
2015-03-17 11:20:20 +00:00
CodingCat 25f35806e3 [SPARK-4011] tighten the visibility of the members in Master/Worker class
https://issues.apache.org/jira/browse/SPARK-4011

Currently, most of the members in Master/Worker are with public accessibility. We might wish to tighten the accessibility of them

a bit more discussion is here:

https://github.com/apache/spark/pull/2828

Author: CodingCat <zhunansjtu@gmail.com>

Closes #4844 from CodingCat/SPARK-4011 and squashes the following commits:

1a64175 [CodingCat] fix compilation issue
e7fd375 [CodingCat] Sean is right....
f5034a4 [CodingCat] fix rebase mistake
8d5b0c0 [CodingCat] loose more fields
0072f96 [CodingCat] lose some restrictions based on the possible design intention
de77286 [CodingCat] tighten accessibility of deploy package
12b4fd3 [CodingCat] tighten accessibility of deploy.worker
1243bc7 [CodingCat] tighten accessibility of deploy.rest
c5f622c [CodingCat] tighten the accessibility of deploy.history
d441e20 [CodingCat] tighten accessibility of deploy.client
4e0ce4a [CodingCat] tighten the accessibility of the members of classes in master
23cddbb [CodingCat] stylistic fix
9a3a340 [CodingCat] tighten the access of worker class
67a0559 [CodingCat] tighten the access permission in Master
2015-03-17 11:18:27 +00:00
Sean Owen b2d8c02224 SPARK-6044 [CORE] RDD.aggregate() should not use the closure serializer on the zero value
Use configured serializer in RDD.aggregate, to match PairRDDFunctions.aggregateByKey, instead of closure serializer.

Compare with e60ad2f4c4/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala (L127)

Author: Sean Owen <sowen@cloudera.com>

Closes #5028 from srowen/SPARK-6044 and squashes the following commits:

a4040a7 [Sean Owen] Use configured serializer in RDD.aggregate, to match PairRDDFunctions.aggregateByKey, instead of closure serializer
2015-03-16 23:58:52 -07:00
Kevin (Sangwoo) Kim f0edeae7f9 [SPARK-6299][CORE] ClassNotFoundException in standalone mode when running groupByKey with class defined in REPL
```
case class ClassA(value: String)
val rdd = sc.parallelize(List(("k1", ClassA("v1")), ("k1", ClassA("v2")) ))
rdd.groupByKey.collect
```
This code used to be throw exception in spark-shell, because while shuffling ```JavaSerializer```uses ```defaultClassLoader``` which was defined like ```env.serializer.setDefaultClassLoader(urlClassLoader)```.

It should be ```env.serializer.setDefaultClassLoader(replClassLoader)```, like
```
    override def run() {
      val deserializeStartTime = System.currentTimeMillis()
      Thread.currentThread.setContextClassLoader(replClassLoader)
```
in TaskRunner.

When ```replClassLoader``` cannot be defined, it's identical with ```urlClassLoader```

Author: Kevin (Sangwoo) Kim <sangwookim.me@gmail.com>

Closes #5046 from swkimme/master and squashes the following commits:

fa2b9ee [Kevin (Sangwoo) Kim] stylish test codes ( collect -> collect() )
6e9620b [Kevin (Sangwoo) Kim] stylish test codes ( collect -> collect() )
d23e4e2 [Kevin (Sangwoo) Kim] stylish test codes ( collect -> collect() )
a4a3c8a [Kevin (Sangwoo) Kim] add 'class defined in repl - shuffle' test to ReplSuite
bd00da5 [Kevin (Sangwoo) Kim] add 'class defined in repl - shuffle' test to ReplSuite
c1b1fc7 [Kevin (Sangwoo) Kim] use REPL class loader for executor's serializer
2015-03-16 23:49:23 -07:00
lisurprise f149b8b5e5 [SPARK-6077] Remove streaming tab while stopping StreamingContext
Currently we would create a new streaming tab for each streamingContext even if there's already one on the same sparkContext which would cause duplicate StreamingTab created and none of them is taking effect.
snapshot: https://www.dropbox.com/s/t4gd6hqyqo0nivz/bad%20multiple%20streamings.png?dl=0
How to reproduce:
1)
import org.apache.spark.SparkConf
import org.apache.spark.streaming.
{Seconds, StreamingContext}
import org.apache.spark.storage.StorageLevel
val ssc = new StreamingContext(sc, Seconds(1))
val lines = ssc.socketTextStream("localhost", 9999, StorageLevel.MEMORY_AND_DISK_SER)
val words = lines.flatMap(_.split(" "))
val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
wordCounts.print()
ssc.start()
.....
2)
ssc.stop(false)
val ssc = new StreamingContext(sc, Seconds(1))
val lines = ssc.socketTextStream("localhost", 9999, StorageLevel.MEMORY_AND_DISK_SER)
val words = lines.flatMap(_.split(" "))
val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
wordCounts.print()
ssc.start()

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

Closes #4828 from zhichao-li/master and squashes the following commits:

c329806 [lisurprise] add test for attaching/detaching streaming tab
51e6c7f [lisurprise] move detach method into StreamingTab
31a44fa [lisurprise] add unit test for attaching and detaching new tab
db25ed2 [lisurprise] clean code
8281bcb [lisurprise] clean code
193c542 [lisurprise] remove streaming tab while closing streaming context
2015-03-16 13:10:32 -07:00
DoingDone9 00e730b94c [SPARK-6300][Spark Core] sc.addFile(path) does not support the relative path.
when i run cmd like that sc.addFile("../test.txt"), it did not work and throwed an exception:
java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: file:../test.txt
at org.apache.hadoop.fs.Path.initialize(Path.java:206)
at org.apache.hadoop.fs.Path.<init>(Path.java:172)
........
.......
Caused by: java.net.URISyntaxException: Relative path in absolute URI: file:../test.txt
at java.net.URI.checkPath(URI.java:1804)
at java.net.URI.<init>(URI.java:752)
at org.apache.hadoop.fs.Path.initialize(Path.java:203)

Author: DoingDone9 <799203320@qq.com>

Closes #4993 from DoingDone9/relativePath and squashes the following commits:

ee375cd [DoingDone9] Update SparkContextSuite.scala
d594e16 [DoingDone9] Update SparkContext.scala
0ff3fa8 [DoingDone9] test for add file
dced8eb [DoingDone9] Update SparkContext.scala
e4a13fe [DoingDone9] getCanonicalPath
161cae3 [DoingDone9] Merge pull request #4 from apache/master
c87e8b6 [DoingDone9] Merge pull request #3 from apache/master
cb1852d [DoingDone9] Merge pull request #2 from apache/master
c3f046f [DoingDone9] Merge pull request #1 from apache/master
2015-03-16 12:27:15 +00:00
Wenchen Fan ea3d2eed9b [CORE][minor] remove unnecessary ClassTag in DAGScheduler
This existed at the very beginning, but became unnecessary after [this commit](37d8f37a8e (diff-6a9ff7fb74fd490a50462d45db2d5e11L272)). I think we should remove it if we don't plan to use it in the future.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #4992 from cloud-fan/small and squashes the following commits:

e857f2e [Wenchen Fan] remove unnecessary ClassTag
2015-03-13 14:08:56 +00:00
Zhang, Liye 9048e8102e [SPARK-6197][CORE] handle json exception when hisotry file not finished writing
For details, please refer to [SPARK-6197](https://issues.apache.org/jira/browse/SPARK-6197)

Author: Zhang, Liye <liye.zhang@intel.com>

Closes #4927 from liyezhang556520/jsonParseError and squashes the following commits:

5cbdc82 [Zhang, Liye] without unnecessary wrap
2b48831 [Zhang, Liye] small changes with sean owen's comments
2973024 [Zhang, Liye] handle json exception when file not finished writing
2015-03-13 14:00:45 +00:00
Ilya Ganelin 0af9ea74a0 [SPARK-5845][Shuffle] Time to cleanup spilled shuffle files not included in shuffle write time
I've added a timer in the right place to fix this inaccuracy.

Author: Ilya Ganelin <ilya.ganelin@capitalone.com>

Closes #4965 from ilganeli/SPARK-5845 and squashes the following commits:

bfabf88 [Ilya Ganelin] Changed to using a foreach vs. getorelse
3e059b0 [Ilya Ganelin] Switched to using getorelse
b946d08 [Ilya Ganelin] Fixed error with option
9434b50 [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5845
db8647e [Ilya Ganelin] Added update for shuffleWriteTime around spilled file cleanup in ExternalSorter
2015-03-13 13:21:04 +00:00
Davies Liu 712679a7b4 [SPARK-6294] fix hang when call take() in JVM on PythonRDD
The Thread.interrupt() can not terminate the thread in some cases, so we should not wait for the writerThread of PythonRDD.

This PR also ignore some exception during clean up.

cc JoshRosen mengxr

Author: Davies Liu <davies@databricks.com>

Closes #4987 from davies/fix_take and squashes the following commits:

4488f1a [Davies Liu] fix hang when call take() in JVM on PythonRDD
2015-03-12 01:34:38 -07:00
Sean Owen 6e94c4eadf SPARK-6225 [CORE] [SQL] [STREAMING] Resolve most build warnings, 1.3.0 edition
Resolve javac, scalac warnings of various types -- deprecations, Scala lang, unchecked cast, etc.

Author: Sean Owen <sowen@cloudera.com>

Closes #4950 from srowen/SPARK-6225 and squashes the following commits:

3080972 [Sean Owen] Ordered imports: Java, Scala, 3rd party, Spark
c67985b [Sean Owen] Resolve javac, scalac warnings of various types -- deprecations, Scala lang, unchecked cast, etc.
2015-03-11 13:15:19 +00:00
Marcelo Vanzin 517975d89d [SPARK-4924] Add a library for launching Spark jobs programmatically.
This change encapsulates all the logic involved in launching a Spark job
into a small Java library that can be easily embedded into other applications.

The overall goal of this change is twofold, as described in the bug:

- Provide a public API for launching Spark processes. This is a common request
  from users and currently there's no good answer for it.

- Remove a lot of the duplicated code and other coupling that exists in the
  different parts of Spark that deal with launching processes.

A lot of the duplication was due to different code needed to build an
application's classpath (and the bootstrapper needed to run the driver in
certain situations), and also different code needed to parse spark-submit
command line options in different contexts. The change centralizes those
as much as possible so that all code paths can rely on the library for
handling those appropriately.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #3916 from vanzin/SPARK-4924 and squashes the following commits:

18c7e4d [Marcelo Vanzin] Fix make-distribution.sh.
2ce741f [Marcelo Vanzin] Add lots of quotes.
3b28a75 [Marcelo Vanzin] Update new pom.
a1b8af1 [Marcelo Vanzin] Merge branch 'master' into SPARK-4924
897141f [Marcelo Vanzin] Review feedback.
e2367d2 [Marcelo Vanzin] Merge branch 'master' into SPARK-4924
28cd35e [Marcelo Vanzin] Remove stale comment.
b1d86b0 [Marcelo Vanzin] Merge branch 'master' into SPARK-4924
00505f9 [Marcelo Vanzin] Add blurb about new API in the programming guide.
5f4ddcc [Marcelo Vanzin] Better usage messages.
92a9cfb [Marcelo Vanzin] Fix Win32 launcher, usage.
6184c07 [Marcelo Vanzin] Rename field.
4c19196 [Marcelo Vanzin] Update comment.
7e66c18 [Marcelo Vanzin] Fix pyspark tests.
0031a8e [Marcelo Vanzin] Review feedback.
c12d84b [Marcelo Vanzin] Review feedback. And fix spark-submit on Windows.
e2d4d71 [Marcelo Vanzin] Simplify some code used to launch pyspark.
43008a7 [Marcelo Vanzin] Don't make builder extend SparkLauncher.
b4d6912 [Marcelo Vanzin] Use spark-submit script in SparkLauncher.
28b1434 [Marcelo Vanzin] Add a comment.
304333a [Marcelo Vanzin] Fix propagation of properties file arg.
bb67b93 [Marcelo Vanzin] Remove unrelated Yarn change (that is also wrong).
8ec0243 [Marcelo Vanzin] Add missing newline.
95ddfa8 [Marcelo Vanzin] Fix handling of --help for spark-class command builder.
72da7ec [Marcelo Vanzin] Rename SparkClassLauncher.
62978e4 [Marcelo Vanzin] Minor cleanup of Windows code path.
9cd5b44 [Marcelo Vanzin] Make all non-public APIs package-private.
e4c80b6 [Marcelo Vanzin] Reorganize the code so that only SparkLauncher is public.
e50dc5e [Marcelo Vanzin] Merge branch 'master' into SPARK-4924
de81da2 [Marcelo Vanzin] Fix CommandUtils.
86a87bf [Marcelo Vanzin] Merge branch 'master' into SPARK-4924
2061967 [Marcelo Vanzin] Merge branch 'master' into SPARK-4924
46d46da [Marcelo Vanzin] Clean up a test and make it more future-proof.
b93692a [Marcelo Vanzin] Merge branch 'master' into SPARK-4924
ad03c48 [Marcelo Vanzin] Revert "Fix a thread-safety issue in "local" mode."
0b509d0 [Marcelo Vanzin] Merge branch 'master' into SPARK-4924
23aa2a9 [Marcelo Vanzin] Read java-opts from conf dir, not spark home.
7cff919 [Marcelo Vanzin] Javadoc updates.
eae4d8e [Marcelo Vanzin] Fix new unit tests on Windows.
e570fb5 [Marcelo Vanzin] Merge branch 'master' into SPARK-4924
44cd5f7 [Marcelo Vanzin] Add package-info.java, clean up javadocs.
f7cacff [Marcelo Vanzin] Remove "launch Spark in new thread" feature.
7ed8859 [Marcelo Vanzin] Some more feedback.
54cd4fd [Marcelo Vanzin] Merge branch 'master' into SPARK-4924
61919df [Marcelo Vanzin] Clean leftover debug statement.
aae5897 [Marcelo Vanzin] Use launcher classes instead of jars in non-release mode.
e584fc3 [Marcelo Vanzin] Rework command building a little bit.
525ef5b [Marcelo Vanzin] Rework Unix spark-class to handle argument with newlines.
8ac4e92 [Marcelo Vanzin] Minor test cleanup.
e946a99 [Marcelo Vanzin] Merge PySparkLauncher into SparkSubmitCliLauncher.
c617539 [Marcelo Vanzin] Review feedback round 1.
fc6a3e2 [Marcelo Vanzin] Merge branch 'master' into SPARK-4924
f26556b [Marcelo Vanzin] Fix a thread-safety issue in "local" mode.
2f4e8b4 [Marcelo Vanzin] Changes needed to make this work with SPARK-4048.
799fc20 [Marcelo Vanzin] Merge branch 'master' into SPARK-4924
bb5d324 [Marcelo Vanzin] Merge branch 'master' into SPARK-4924
53faef1 [Marcelo Vanzin] Merge branch 'master' into SPARK-4924
a7936ef [Marcelo Vanzin] Fix pyspark tests.
656374e [Marcelo Vanzin] Mima fixes.
4d511e7 [Marcelo Vanzin] Fix tools search code.
7a01e4a [Marcelo Vanzin] Fix pyspark on Yarn.
1b3f6e9 [Marcelo Vanzin] Call SparkSubmit from spark-class launcher for unknown classes.
25c5ae6 [Marcelo Vanzin] Centralize SparkSubmit command line parsing.
27be98a [Marcelo Vanzin] Modify Spark to use launcher lib.
6f70eea [Marcelo Vanzin] [SPARK-4924] Add a library for launching Spark jobs programatically.
2015-03-11 01:03:01 -07:00
Lev Khomich c4c4b07bf6 [SPARK-6087][CORE] Provide actionable exception if Kryo buffer is not large enough
A simple try-catch wrapping KryoException to be more informative.

Author: Lev Khomich <levkhomich@gmail.com>

Closes #4947 from levkhomich/master and squashes the following commits:

0f7a947 [Lev Khomich] [SPARK-6087][CORE] Provide actionable exception if Kryo buffer is not large enough
2015-03-10 10:55:42 +00:00
Davies Liu 8767565cef [SPARK-6194] [SPARK-677] [PySpark] fix memory leak in collect()
Because circular reference between JavaObject and JavaMember, an Java object can not be released until Python GC kick in, then it will cause memory leak in collect(), which may consume lots of memory in JVM.

This PR change the way we sending collected data back into Python from local file to socket, which could avoid any disk IO during collect, also avoid any referrers of Java object in Python.

cc JoshRosen

Author: Davies Liu <davies@databricks.com>

Closes #4923 from davies/fix_collect and squashes the following commits:

d730286 [Davies Liu] address comments
24c92a4 [Davies Liu] fix style
ba54614 [Davies Liu] use socket to transfer data from JVM
9517c8f [Davies Liu] fix memory leak in collect()
2015-03-09 16:24:06 -07:00
Sean Owen f16b7b031f SPARK-6205 [CORE] UISeleniumSuite fails for Hadoop 2.x test with NoClassDefFoundError
Add xml-apis to core test deps to work aroudn UISeleniumSuite classpath issue

Author: Sean Owen <sowen@cloudera.com>

Closes #4933 from srowen/SPARK-6205 and squashes the following commits:

ddd4d32 [Sean Owen] Add xml-apis to core test deps to work aroudn UISeleniumSuite classpath issue
2015-03-08 14:09:40 +00:00
GuoQiang Li 05cb6b34d8 [Minor] Resolve sbt warnings: postfix operator second should be enabled
Resolve sbt warnings:

```
[warn] spark/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala:155: postfix operator second should be enabled
[warn] by making the implicit value scala.language.postfixOps visible.
[warn] This can be achieved by adding the import clause 'import scala.language.postfixOps'
[warn] or by setting the compiler option -language:postfixOps.
[warn] See the Scala docs for value scala.language.postfixOps for a discussion
[warn] why the feature should be explicitly enabled.
[warn]         Await.ready(f, 1 second)
[warn]                          ^
```

Author: GuoQiang Li <witgo@qq.com>

Closes #4908 from witgo/sbt_warnings and squashes the following commits:

0629af4 [GuoQiang Li] Resolve sbt warnings: postfix operator second should be enabled
2015-03-06 13:20:20 +00:00
Marcelo Vanzin cd7594ca6a [core] [minor] Don't pollute source directory when running UtilsSuite.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #4921 from vanzin/utils-suite and squashes the following commits:

7795dd4 [Marcelo Vanzin] [core] [minor] Don't pollute source directory when running UtilsSuite.
2015-03-06 09:43:24 +00:00
Josh Rosen 424a86a1ed [SPARK-6175] Fix standalone executor log links when ephemeral ports or SPARK_PUBLIC_DNS are used
This patch fixes two issues with the executor log viewing links added in Spark 1.3.  In standalone mode, the log URLs might include a port value of 0 rather than the actual bound port of the UI, which broke the ability to view logs from workers whose web UIs had been configured to bind to ephemeral ports.  In addition, the URLs used workers' local hostnames instead of respecting SPARK_PUBLIC_DNS, which prevented this feature from working properly on Spark EC2 clusters because the links would point to internal DNS names instead of external ones.

I included tests for both of these bugs:

- We now browse to the URLs and verify that they point to the expected pages.
- To test SPARK_PUBLIC_DNS, I changed the code that reads the environment variable to do so via `SparkConf.getenv`, then used a custom SparkConf subclass to mock the environment variable (this pattern is used elsewhere in Spark's tests).

Author: Josh Rosen <joshrosen@databricks.com>

Closes #4903 from JoshRosen/SPARK-6175 and squashes the following commits:

5577f41 [Josh Rosen] Remove println
cfec135 [Josh Rosen] Use webUi.boundPort and publicAddress in log links
27918c7 [Josh Rosen] Add failing unit tests for standalone log URL viewing
c250fbe [Josh Rosen] Respect SparkConf in local-cluster Workers.
422a2ef [Josh Rosen] Use conf.getenv to read SPARK_PUBLIC_DNS
2015-03-05 12:04:00 -08:00
Sean Owen c9cfba0ceb SPARK-6182 [BUILD] spark-parent pom needs to be published for both 2.10 and 2.11
Option 1 of 2: Convert spark-parent module name to spark-parent_2.10 / spark-parent_2.11

Author: Sean Owen <sowen@cloudera.com>

Closes #4912 from srowen/SPARK-6182.1 and squashes the following commits:

eff60de [Sean Owen] Convert spark-parent module name to spark-parent_2.10 / spark-parent_2.11
2015-03-05 11:31:48 -08:00
Marcelo Vanzin 3a35a0dfe9 [SPARK-6144] [core] Fix addFile when source files are on "hdfs:"
The code failed in two modes: it complained when it tried to re-create a directory that already existed, and it was placing some files in the wrong parent directory. The patch fixes both issues.

Author: Marcelo Vanzin <vanzin@cloudera.com>
Author: trystanleftwich <trystan@atscale.com>

Closes #4894 from vanzin/SPARK-6144 and squashes the following commits:

100b3a1 [Marcelo Vanzin] Style fix.
58266aa [Marcelo Vanzin] Fix fetchHcfs file for directories.
91733b7 [trystanleftwich] [SPARK-6144]When in cluster mode using ADD JAR with a hdfs:// sourced jar will fail
2015-03-04 12:58:39 -08:00
Zhang, Liye f6773edce0 [SPARK-6107][CORE] Display inprogress application information for event log history for standalone mode
when application is finished running abnormally (Ctrl + c for example), the history event log file is still ends with `.inprogress` suffix. And the application state can not be showed on webUI, User can only see "*Application history not foud xxxx, Application xxx is still in progress*".

For application that not finished normally, the history will show:
![image](https://cloud.githubusercontent.com/assets/4716022/6437137/184f9fc0-c0f5-11e4-88cc-a2eb087e4561.png)

Author: Zhang, Liye <liye.zhang@intel.com>

Closes #4848 from liyezhang556520/showLogInprogress and squashes the following commits:

03589ac [Zhang, Liye] change inprogress to in progress
b55f19f [Zhang, Liye] scala modify after rebase
8aa66a2 [Zhang, Liye] use softer wording
b030bd4 [Zhang, Liye] clean code
79c8cb1 [Zhang, Liye] fix some mistakes
11cdb68 [Zhang, Liye] add a missing space
c29205b [Zhang, Liye] refine code according to sean owen's comments
e9952a7 [Zhang, Liye] scala style fix again
150502d [Zhang, Liye] scala style fix
f11a5da [Zhang, Liye] small fix for file path
22e878b [Zhang, Liye] enable in progress eventlog file
2015-03-04 12:28:27 +00:00
tedyu 8d3e2414d4 SPARK-6085 Increase default value for memory overhead
Author: tedyu <yuzhihong@gmail.com>

Closes #4836 from tedyu/master and squashes the following commits:

d65b495 [tedyu] SPARK-6085 Increase default value for memory overhead
1fdd4df [tedyu] SPARK-6085 Increase default value for memory overhead
2015-03-04 11:00:52 +00:00
Andrew Or d334bfbcf3 [SPARK-6132][HOTFIX] ContextCleaner InterruptedException should be quiet
If the cleaner is stopped, we shouldn't print a huge stack trace when the cleaner thread is interrupted because we purposefully did this.

Author: Andrew Or <andrew@databricks.com>

Closes #4882 from andrewor14/cleaner-interrupt and squashes the following commits:

8652120 [Andrew Or] Just a hot fix
2015-03-03 20:49:45 -08:00
Imran Rashid 1f1fccc5ce [SPARK-5949] HighlyCompressedMapStatus needs more classes registered w/ kryo
https://issues.apache.org/jira/browse/SPARK-5949

Author: Imran Rashid <irashid@cloudera.com>

Closes #4877 from squito/SPARK-5949_register_roaring_bitmap and squashes the following commits:

7e13316 [Imran Rashid] style style style
5f6bb6d [Imran Rashid] more style
709bfe0 [Imran Rashid] style
a5cb744 [Imran Rashid] update tests to cover both types of RoaringBitmapContainers
09610c6 [Imran Rashid] formatting
f9a0b7c [Imran Rashid] put primitive array registrations together
97beaf8 [Imran Rashid] SPARK-5949 HighlyCompressedMapStatus needs more classes registered w/ kryo
2015-03-03 15:33:19 -08:00
Andrew Or 6c20f35290 [SPARK-6133] Make sc.stop() idempotent
Before we would get the following (benign) error if we called `sc.stop()` twice. This is because the listener bus would try to post the end event again even after it has already stopped. This happens occasionally when flaky tests fail, usually as a result of other sources of error. Either way we shouldn't be logging this error when it is not the cause of the failure.
```
ERROR LiveListenerBus: SparkListenerBus has already stopped! Dropping event SparkListenerApplicationEnd(1425348445682)
```

Author: Andrew Or <andrew@databricks.com>

Closes #4871 from andrewor14/sc-stop and squashes the following commits:

a14afc5 [Andrew Or] Move code after code
915db16 [Andrew Or] Move code into code
2015-03-03 15:09:57 -08:00
Andrew Or fe63e82291 [SPARK-6132] ContextCleaner race condition across SparkContexts
The problem is that `ContextCleaner` may clean variables that belong to a different `SparkContext`. This can happen if the `SparkContext` to which the cleaner belongs stops, and a new one is started immediately afterwards in the same JVM. In this case, if the cleaner is in the middle of cleaning a broadcast, for instance, it will do so through `SparkEnv.get.blockManager`, which could be one that belongs to a different `SparkContext`.

JoshRosen and I suspect that this is the cause of many flaky tests, most notably the `JavaAPISuite`. We were able to reproduce the failure locally (though it is not deterministic and very hard to reproduce).

Author: Andrew Or <andrew@databricks.com>

Closes #4869 from andrewor14/cleaner-masquerade and squashes the following commits:

29168c0 [Andrew Or] Synchronize ContextCleaner stop
2015-03-03 13:44:05 -08:00
Andrew Or 9af001749a Revert "[SPARK-5423][Core] Cleanup resources in DiskMapIterator.finalize to ensure deleting the temp file"
This reverts commit 90095bf3ce.
2015-03-03 13:03:52 -08:00
Wenchen Fan e359794cec [SPARK-6138][CORE][minor] enhance the toArray method in SizeTrackingVector
Use array copy instead of `Iterator#toArray` to make it more efficient.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #4825 from cloud-fan/minor and squashes the following commits:

c933ee5 [Wenchen Fan] make toArray method just in parent
946a35b [Wenchen Fan] minor enhance
2015-03-03 12:12:23 +00:00
CodingCat 975643c256 [SPARK-6118] making package name of deploy.worker.CommandUtils and deploy.CommandUtilsSuite consistent
https://issues.apache.org/jira/browse/SPARK-6118

I found that the object CommandUtils is placed under deploy.worker package, while CommandUtilsSuite is  under deploy

Conventionally, we put the implementation and unit test class under the same package

here, to minimize the change, I move CommandUtilsSuite to worker package,

**However, CommandUtils seems to contain some general methods (though only used by worker.* classes currently**,  we may also consider to replace CommonUtils

Author: CodingCat <zhunansjtu@gmail.com>

Closes #4856 from CodingCat/SPARK-6118 and squashes the following commits:

cb93700 [CodingCat] making package name consistent
2015-03-03 10:32:57 +00:00
hushan[胡珊] e3a88d1104 [SPARK-4777][CORE] Some block memory after unrollSafely not count into used memory(memoryStore.entrys or unrollMemory)
Some memory not count into memory used by memoryStore or unrollMemory.
Thread A after unrollsafely memory, it will release 40MB unrollMemory(40MB will used by other threads). then ThreadA wait get accountingLock to tryToPut blockA(30MB). before Thread A get accountingLock, blockA memory size is not counting into unrollMemory or memoryStore.currentMemory.
IIUC, freeMemory should minus that block memory

So, put this release memory into pending, and release it in tryToPut before ensureSpace

Author: hushan[胡珊] <hushan@xiaomi.com>

Closes #3629 from suyanNone/unroll-memory and squashes the following commits:

809cc41 [hushan[胡珊]] Refine
407b2c9 [hushan[胡珊]] Refine according comments
39960d0 [hushan[胡珊]] Refine comments
0fd0213 [hushan[胡珊]] add comments
0fc2bec [hushan[胡珊]] Release pending unroll memory after put block in memoryStore
3a3f2c8 [hushan[胡珊]] Refine blockManagerSuite unroll test
3323c45 [hushan[胡珊]] Refine getOrElse
f664317 [hushan[胡珊]] Make sure not add pending in every releaseUnrollMemory call
08b32ba [hushan[胡珊]] Pending unroll memory for this block untill tryToPut
2015-03-02 16:53:54 -08:00
Andrew Or 258d154c9f [SPARK-6048] SparkConf should not translate deprecated configs on set
There are multiple issues with translating on set outlined in the JIRA.

This PR reverts the translation logic added to `SparkConf`. In the future, after the 1.3.0 release we will figure out a way to reorganize the internal structure more elegantly. For now, let's preserve the existing semantics of `SparkConf` since it's a public interface. Unfortunately this means duplicating some code for now, but this is all internal and we can always clean it up later.

Author: Andrew Or <andrew@databricks.com>

Closes #4799 from andrewor14/conf-set-translate and squashes the following commits:

11c525b [Andrew Or] Move warning to driver
10e77b5 [Andrew Or] Add documentation for deprecation precedence
a369cb1 [Andrew Or] Merge branch 'master' of github.com:apache/spark into conf-set-translate
c26a9e3 [Andrew Or] Revert all translate logic in SparkConf
fef6c9c [Andrew Or] Restore deprecation logic for spark.executor.userClassPathFirst
94b4dfa [Andrew Or] Translate on get, not set
2015-03-02 16:36:42 -08:00
Andrew Or 6776cb33ea [SPARK-6066] Make event log format easier to parse
Some users have reported difficulty in parsing the new event log format. Since we embed the metadata in the beginning of the file, when we compress the event log we need to skip the metadata because we need that information to parse the log later. This means we'll end up with a partially compressed file if event logging compression is turned on. The old format looks like:
```
sparkVersion = 1.3.0
compressionCodec = org.apache.spark.io.LZFCompressionCodec
=== LOG_HEADER_END ===
// actual events, could be compressed bytes
```
The new format in this patch puts the compression codec in the log file name instead. It also removes the metadata header altogether along with the Spark version, which was not needed. The new file name looks something like:
```
app_without_compression
app_123.lzf
app_456.snappy
```

I tested this with and without compression, using different compression codecs and event logging directories. I verified that both the `Master` and the `HistoryServer` can render both compressed and uncompressed logs as before.

Author: Andrew Or <andrew@databricks.com>

Closes #4821 from andrewor14/event-log-format and squashes the following commits:

8511141 [Andrew Or] Fix test
654883d [Andrew Or] Add back metadata with Spark version
7f537cd [Andrew Or] Address review feedback
7d6aa61 [Andrew Or] Make codec an extension
59abee9 [Andrew Or] Merge branch 'master' of github.com:apache/spark into event-log-format
27c9a6c [Andrew Or] Address review feedback
519e51a [Andrew Or] Address review feedback
ef69276 [Andrew Or] Merge branch 'master' of github.com:apache/spark into event-log-format
88a091d [Andrew Or] Add tests for new format and file name
f32d8d2 [Andrew Or] Fix tests
8db5a06 [Andrew Or] Embed metadata in the event log file name instead
2015-03-02 16:34:32 -08:00
guliangliang 26c1c56dea [SPARK-5522] Accelerate the Histroty Server start
When starting the history server, all the log files will be fetched and parsed in order to get the applications' meta data e.g. App Name, Start Time, Duration, etc. In our production cluster, there exist 2600 log files (160G) in HDFS and it costs 3 hours to restart the history server, which is a little bit too long for us.

It would be better, if the history server can show logs with missing information during start-up and fill the missing information after fetching and parsing a log file.

Author: guliangliang <guliangliang@qiyi.com>

Closes #4525 from marsishandsome/Spark5522 and squashes the following commits:

a865c11 [guliangliang] fix bug2
4340c2b [guliangliang] fix bug
af92a5a [guliangliang] [SPARK-5522] Accelerate the Histroty Server start
2015-03-02 15:33:23 -08:00
Sean Owen 948c2390ab SPARK-3357 [CORE] Internal log messages should be set at DEBUG level instead of INFO
Demote some 'noisy' log messages to debug level. I added a few more, to include everything that gets logged in stanzas like this:

```
15/03/01 00:03:54 INFO BlockManager: Removing broadcast 0
15/03/01 00:03:54 INFO BlockManager: Removing block broadcast_0_piece0
15/03/01 00:03:54 INFO MemoryStore: Block broadcast_0_piece0 of size 839 dropped from memory (free 277976091)
15/03/01 00:03:54 INFO BlockManagerInfo: Removed broadcast_0_piece0 on localhost:49524 in memory (size: 839.0 B, free: 265.1 MB)
15/03/01 00:03:54 INFO BlockManagerMaster: Updated info of block broadcast_0_piece0
15/03/01 00:03:54 INFO BlockManager: Removing block broadcast_0
15/03/01 00:03:54 INFO MemoryStore: Block broadcast_0 of size 1088 dropped from memory (free 277977179)
15/03/01 00:03:54 INFO ContextCleaner: Cleaned broadcast 0
```

as well as regular messages like

```
15/03/01 00:02:33 INFO MemoryStore: ensureFreeSpace(2640) called with curMem=47322, maxMem=278019440
```

WDYT? good or should some be left alone?

CC mengxr who suggested some of this.

Author: Sean Owen <sowen@cloudera.com>

Closes #4838 from srowen/SPARK-3357 and squashes the following commits:

dce75c1 [Sean Owen] Back out some debug level changes
d9b784d [Sean Owen] Demote some 'noisy' log messages to debug level
2015-03-02 08:51:03 +00:00
Josh Rosen 2df5f1f006 [SPARK-6075] Fix bug in that caused lost accumulator updates: do not store WeakReferences in localAccums map
This fixes a non-deterministic bug introduced in #4021 that could cause tasks' accumulator updates to be lost.  The problem is that `localAccums` should not hold weak references: after the task finishes running there won't be any strong references to these local accumulators, so they can get garbage-collected before the executor reads the `localAccums` map.  We don't need weak references here anyways, since this map is cleared at the end of each task.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #4835 from JoshRosen/SPARK-6075 and squashes the following commits:

4f4b5b2 [Josh Rosen] Remove defensive assertions that caused test failures in code unrelated to this change
120c7b0 [Josh Rosen] [SPARK-6075] Do not store WeakReferences in localAccums map
2015-02-28 22:51:01 -08:00
Evan Yu 643300a6e2 SPARK-5984: Fix TimSort bug causes ArrayOutOfBoundsException
Fix TimSort bug which causes a ArrayOutOfBoundsException.

Using the proposed fix here
http://envisage-project.eu/proving-android-java-and-python-sorting-algorithm-is-broken-and-how-to-fix-it/

Author: Evan Yu <ehotou@gmail.com>

Closes #4804 from hotou/SPARK-5984 and squashes the following commits:

3421b6c [Evan Yu] SPARK-5984: Add info to LICENSE
e61c6b8 [Evan Yu] SPARK-5984: Fix license and document
6ccc280 [Evan Yu] SPARK-5984: Add License header to file
e06c0d2 [Evan Yu] SPARK-5984: Add License header to file
4d95f75 [Evan Yu] SPARK-5984: Fix TimSort bug causes ArrayOutOfBoundsException
479a106 [Evan Yu] SPARK-5984: Fix TimSort bug causes ArrayOutOfBoundsException
2015-02-28 18:55:34 -08:00
Sean Owen 86fcdaef62 SPARK-1965 [WEBUI] Spark UI throws NPE on trying to load the app page for non-existent app
Don't throw NPE if appId is unknown. kayousterhout is this a decent enough band-aid for avoiding a full-blown NPE? it should just render empty content instead

Author: Sean Owen <sowen@cloudera.com>

Closes #4777 from srowen/SPARK-1965 and squashes the following commits:

7e16590 [Sean Owen] Update app not found message
cb878d6 [Sean Owen] Return basic "not found" page for unknown appId
d8270da [Sean Owen] Don't throw NPE if appId is unknown
2015-02-28 15:34:08 +00:00
Sean Owen f91298e2c5 SPARK-5983 [WEBUI] Don't respond to HTTP TRACE in HTTP-based UIs
Disallow TRACE HTTP method in servlets

Author: Sean Owen <sowen@cloudera.com>

Closes #4765 from srowen/SPARK-5983 and squashes the following commits:

421b25b [Sean Owen] Disallow TRACE HTTP method in servlets
2015-02-28 15:23:59 +00:00
Burak Yavuz 6d8e5fbc0d [SPARK-5979][SPARK-6032] Smaller safer --packages fix
pwendell tdas
This is the safer parts of PR #4754:
 - SPARK-5979: All dependencies with the groupId `org.apache.spark` passed through `--packages`, were being excluded from the dependency tree on the assumption that they would be in the assembly jar. This is not the case, therefore the exclusion rules had to be defined more explicitly.
 - SPARK-6032: Ivy prints a whole lot of logs while retrieving dependencies. These were printed to `System.out`. Moved the logging to `System.err`.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #4802 from brkyvz/simple-streaming-fix and squashes the following commits:

e0f38cb [Burak Yavuz] Merge branch 'master' of github.com:apache/spark into simple-streaming-fix
bad921c [Burak Yavuz] [SPARK-5979][SPARK-6032] Smaller safer fix
2015-02-27 22:59:35 -08:00
Zhang, Liye 8cd1692c90 [SPARK-6036][CORE] avoid race condition between eventlogListener and akka actor system
For detail description, pls refer to [SPARK-6036](https://issues.apache.org/jira/browse/SPARK-6036).

Author: Zhang, Liye <liye.zhang@intel.com>

Closes #4785 from liyezhang556520/EventLogInProcess and squashes the following commits:

8b0b0a6 [Zhang, Liye] stop listener after DAGScheduler
79b15b3 [Zhang, Liye] SPARK-6036 avoid race condition between eventlogListener and akka actor system
2015-02-26 23:11:43 -08:00
Andrew Or 7c99a014fb [SPARK-6046] Privatize SparkConf.translateConfKey
The warning of deprecated configs is actually done when the configs are set, not when they are get. As a result we don't need to explicitly call `translateConfKey` outside of `SparkConf` just to print the warning again in vain.

Author: Andrew Or <andrew@databricks.com>

Closes #4797 from andrewor14/warn-deprecated-config and squashes the following commits:

8fb43e6 [Andrew Or] Privatize SparkConf.translateConfKey
2015-02-26 22:39:46 -08:00
Lukasz Jastrzebski 4a8a0a8ecd SPARK-2168 [Spark core] Use relative URIs for the app links in the History Server.
As agreed in PR #1160 adding test to verify if history server generates relative links to applications.

Author: Lukasz Jastrzebski <lukasz.jastrzebski@gmail.com>

Closes #4778 from elyast/master and squashes the following commits:

0c07fab [Lukasz Jastrzebski] Incorporating comments for SPARK-2168
6d7866d [Lukasz Jastrzebski] Adjusting test for  SPARK-2168 for master branch
d6f4fbe [Lukasz Jastrzebski] Added test for  SPARK-2168
2015-02-26 22:38:06 -08:00
jerryshao 67595eb8fb [SPARK-5495][UI] Add app and driver kill function in master web UI
Add application kill function in master web UI for standalone mode. Details can be seen in [SPARK-5495](https://issues.apache.org/jira/browse/SPARK-5495).

The snapshot of UI shows as below:
![snapshot](https://dl.dropboxusercontent.com/u/19230832/master_ui.png)

Please help to review, thanks a lot.

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

Closes #4288 from jerryshao/SPARK-5495 and squashes the following commits:

fa3e486 [jerryshao] Add some conditions
9a7be93 [jerryshao] Add kill Driver function
a239776 [jerryshao] Change the code format
ff5195d [jerryshao] Add app kill function in master web UI
2015-02-26 22:36:48 -08:00
jerryshao 12135e9054 [SPARK-5771][UI][hotfix] Change Requested Cores into * if default cores is not set
cc andrewor14, srowen.

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

Closes #4800 from jerryshao/SPARK-5771 and squashes the following commits:

a2483c2 [jerryshao] Change the UI of Requested Cores into * if default cores is not set
2015-02-26 22:35:43 -08:00
Hong Shen 18f2098433 [SPARK-5529][CORE]Add expireDeadHosts in HeartbeatReceiver
If a blockManager has not send heartBeat more than 120s, BlockManagerMasterActor will remove it. But coarseGrainedSchedulerBackend can only remove executor after an DisassociatedEvent.  We should expireDeadHosts at HeartbeatReceiver.

Author: Hong Shen <hongshen@tencent.com>

Closes #4363 from shenh062326/my_change3 and squashes the following commits:

2c9a46a [Hong Shen] Change some code style.
1a042ff [Hong Shen] Change some code style.
2dc456e [Hong Shen] Change some code style.
d221493 [Hong Shen] Fix test failed
7448ac6 [Hong Shen] A minor change in sparkContext and heartbeatReceiver
b904aed [Hong Shen] Fix failed test
52725af [Hong Shen] Remove assert in SparkContext.killExecutors
5bedcb8 [Hong Shen] Remove assert in SparkContext.killExecutors
a858fb5 [Hong Shen] A minor change in HeartbeatReceiver
3e221d9 [Hong Shen] A minor change in HeartbeatReceiver
6bab7aa [Hong Shen] Change a code style.
07952f3 [Hong Shen] Change configs name and code style.
ce9257e [Hong Shen] Fix test failed
bccd515 [Hong Shen] Fix test failed
8e77408 [Hong Shen] Fix test failed
c1dfda1 [Hong Shen] Fix test failed
e197e20 [Hong Shen] Fix test failed
fb5df97 [Hong Shen] Remove ExpireDeadHosts in BlockManagerMessages
b5c0441 [Hong Shen] Remove expireDeadHosts in BlockManagerMasterActor
c922cb0 [Hong Shen] Add expireDeadHosts in HeartbeatReceiver
2015-02-26 18:43:23 -08:00
Sean Owen fbc469473d SPARK-4579 [WEBUI] Scheduling Delay appears negative
Ensure scheduler delay handles unfinished task case, and ensure delay is never negative even due to rounding

Author: Sean Owen <sowen@cloudera.com>

Closes #4796 from srowen/SPARK-4579 and squashes the following commits:

ad6713c [Sean Owen] Ensure scheduler delay handles unfinished task case, and ensure delay is never negative even due to rounding
2015-02-26 17:35:09 -08:00
tedyu e60ad2f4c4 SPARK-6045 RecordWriter should be checked against null in PairRDDFunctio...
...ns#saveAsNewAPIHadoopDataset

Author: tedyu <yuzhihong@gmail.com>

Closes #4794 from tedyu/master and squashes the following commits:

2632a57 [tedyu] SPARK-6045 RecordWriter should be checked against null in PairRDDFunctions#saveAsNewAPIHadoopDataset
2d8d4b1 [tedyu] SPARK-6045 RecordWriter should be checked against null in PairRDDFunctions#saveAsNewAPIHadoopDataset
2015-02-26 23:27:09 +00:00
Sean Owen 3fb53c0298 SPARK-4300 [CORE] Race condition during SparkWorker shutdown
Close appender saving stdout/stderr before destroying process to avoid exception on reading closed input stream.
(This also removes a redundant `waitFor()` although it was harmless)

CC tdas since I think you wrote this method.

Author: Sean Owen <sowen@cloudera.com>

Closes #4787 from srowen/SPARK-4300 and squashes the following commits:

e0cdabf [Sean Owen] Close appender saving stdout/stderr before destroying process to avoid exception on reading closed input stream
2015-02-26 14:08:56 -08:00
xukun 00228947 8942b522d8 [SPARK-3562]Periodic cleanup event logs
Author: xukun 00228947 <xukun.xu@huawei.com>

Closes #4214 from viper-kun/cleaneventlog and squashes the following commits:

7a5b9c5 [xukun 00228947] fix issue
31674ee [xukun 00228947] fix issue
6e3d06b [xukun 00228947] fix issue
373f3b9 [xukun 00228947] fix issue
71782b5 [xukun 00228947] fix issue
5b45035 [xukun 00228947] fix issue
70c28d6 [xukun 00228947] fix issues
adcfe86 [xukun 00228947] Periodic cleanup event logs
2015-02-26 13:24:00 -08:00
Sean Owen cd5c8d7bbd SPARK-4704 [CORE] SparkSubmitDriverBootstrap doesn't flush output
Join on output threads to make sure any lingering output from process reaches stdout, stderr before exiting

CC andrewor14 since I believe he created this section of code

Author: Sean Owen <sowen@cloudera.com>

Closes #4788 from srowen/SPARK-4704 and squashes the following commits:

ad7114e [Sean Owen] Join on output threads to make sure any lingering output from process reaches stdout, stderr before exiting
2015-02-26 12:56:54 -08:00
Davies Liu 7fa960e653 [SPARK-5363] Fix bug in PythonRDD: remove() inside iterator is not safe
Removing elements from a mutable HashSet while iterating over it can cause the
iteration to incorrectly skip over entries that were not removed. If this
happened, PythonRDD would write fewer broadcast variables than the Python
worker was expecting to read, which would cause the Python worker to hang
indefinitely.

Author: Davies Liu <davies@databricks.com>

Closes #4776 from davies/fix_hang and squashes the following commits:

a4384a5 [Davies Liu] fix bug: remvoe() inside iterator is not safe
2015-02-26 11:54:27 -08:00
Marcelo Vanzin df3d559b32 [SPARK-5801] [core] Avoid creating nested directories.
Cache the value of the local root dirs to use for storing local data,
so that the same directories are reused.

Also, to avoid an extra level of nesting, use a different env variable
to propagate the local dirs from the Worker to the executors. And make
the executor directory use a different name.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #4747 from vanzin/SPARK-5801 and squashes the following commits:

e0114e1 [Marcelo Vanzin] Update unit test.
18ee0a7 [Marcelo Vanzin] [SPARK-5801] [core] Avoid creating nested directories.
2015-02-26 17:35:03 +00:00
Judy Nash 51a6f9097b [SPARK-5914] to run spark-submit requiring only user perm on windows
Because windows on-default does not grant read permission to jars except to admin, spark-submit would fail with "ClassNotFound" exception if user runs slave service with only user permission.
This fix is to add read permission to owner of the jar (which would be the slave service account in windows )

Author: Judy Nash <judynash@microsoft.com>

Closes #4742 from judynash/SPARK-5914 and squashes the following commits:

e288e56 [Judy Nash] Fix spacing and refactor code
1de3c0e [Judy Nash] [SPARK-5914] Enable spark-submit to run requiring only user permission on windows
2015-02-26 11:15:27 +00:00
Kay Ousterhout 838a48036c [SPARK-5982] Remove incorrect Local Read Time Metric
This metric is incomplete, because the files are memory mapped, so much of the read from disk occurs later as tasks actually read the file's data.

This should be merged into 1.3, so that we never expose this incorrect metric to users.

CC pwendell ksakellis sryza

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #4749 from kayousterhout/SPARK-5982 and squashes the following commits:

9737b5e [Kay Ousterhout] More fixes
a1eb300 [Kay Ousterhout] Removed one more use of local read time
cf13497 [Kay Ousterhout] [SPARK-5982] Remove incorrectwq Local Read Time Metric
2015-02-25 14:55:24 -08:00
Milan Straka a777c65da9 [SPARK-5970][core] Register directory created in getOrCreateLocalRootDirs for automatic deletion.
As documented in createDirectory, the result of createDirectory is not registered for automatic removal. Currently there are 4 directories left in `/tmp` after just running `pyspark`.

Author: Milan Straka <fox@ucw.cz>

Closes #4759 from foxik/remove-tmp-dirs and squashes the following commits:

280450d [Milan Straka] Use createTempDir in getOrCreateLocalRootDirs...
2015-02-25 21:33:34 +00:00
guliangliang dd077abf2e [SPARK-5771] Number of Cores in Completed Applications of Standalone Master Web Page always be 0 if sc.stop() is called
In Standalone mode, the number of cores in Completed Applications of the Master Web Page will always be zero, if sc.stop() is called.
But the number will always be right, if sc.stop() is not called.
The reason maybe:
after sc.stop() is called, the function removeExecutor of class ApplicationInfo will be called, thus reduce the variable coresGranted to zero. The variable coresGranted is used to display the number of Cores on the Web Page.

Author: guliangliang <guliangliang@qiyi.com>

Closes #4567 from marsishandsome/Spark5771 and squashes the following commits:

694796e [guliangliang] remove duplicate code
a20e390 [guliangliang] change to Cores Using & Requested
0c19c95 [guliangliang] change Cores to Cores (max)
cfbd97d [guliangliang] [SPARK-5771] Number of Cores in Completed Applications of Standalone Master Web Page always be 0 if sc.stop() is called
2015-02-25 14:48:02 +00:00
Judy c5ba975ee8 [Spark-5708] Add Slf4jSink to Spark Metrics
Add Slf4jSink to Spark Metrics using Coda Hale's SlfjReporter.
This sends metrics to log4j, allowing spark users to reuse log4j pipeline for metrics collection.

Reviewed existing unit tests and didn't see any sink-related tests. Please advise on if tests should be added.

Author: Judy <judynash@microsoft.com>
Author: judynash <judynash@microsoft.com>

Closes #4644 from judynash/master and squashes the following commits:

57ef214 [judynash] doc clarification and indent fixes
a751a66 [Judy] Spark-5708: Add Slf4jSink to Spark Metrics
2015-02-24 20:50:16 +00:00
Andrew Or 6d2caa576f [SPARK-5965] Standalone Worker UI displays {{USER_JAR}}
For screenshot see: https://issues.apache.org/jira/browse/SPARK-5965
This was caused by 20a6013106.

Author: Andrew Or <andrew@databricks.com>

Closes #4739 from andrewor14/user-jar-blocker and squashes the following commits:

23c4a9e [Andrew Or] Use right argument
2015-02-24 11:08:07 -08:00
Tathagata Das 64d2c01ff1 [Spark-5967] [UI] Correctly clean JobProgressListener.stageIdToActiveJobIds
Patch should be self-explanatory
pwendell JoshRosen

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

Closes #4741 from tdas/SPARK-5967 and squashes the following commits:

653b5bb [Tathagata Das] Fixed the fix and added test
e2de972 [Tathagata Das] Clear stages which have no corresponding active jobs.
2015-02-24 11:02:47 -08:00
CodingCat 242d49584c [SPARK-5724] fix the misconfiguration in AkkaUtils
https://issues.apache.org/jira/browse/SPARK-5724

In AkkaUtil, we set several failure detector related the parameters as following

```
al akkaConf = ConfigFactory.parseMap(conf.getAkkaConf.toMap[String, String])
      .withFallback(akkaSslConfig).withFallback(ConfigFactory.parseString(
      s"""
      |akka.daemonic = on
      |akka.loggers = [""akka.event.slf4j.Slf4jLogger""]
      |akka.stdout-loglevel = "ERROR"
      |akka.jvm-exit-on-fatal-error = off
      |akka.remote.require-cookie = "$requireCookie"
      |akka.remote.secure-cookie = "$secureCookie"
      |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s
      |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s
      |akka.remote.transport-failure-detector.threshold = $akkaFailureDetector
      |akka.actor.provider = "akka.remote.RemoteActorRefProvider"
      |akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport"
      |akka.remote.netty.tcp.hostname = "$host"
      |akka.remote.netty.tcp.port = $port
      |akka.remote.netty.tcp.tcp-nodelay = on
      |akka.remote.netty.tcp.connection-timeout = $akkaTimeout s
      |akka.remote.netty.tcp.maximum-frame-size = ${akkaFrameSize}B
      |akka.remote.netty.tcp.execution-pool-size = $akkaThreads
      |akka.actor.default-dispatcher.throughput = $akkaBatchSize
      |akka.log-config-on-start = $logAkkaConfig
      |akka.remote.log-remote-lifecycle-events = $lifecycleEvents
      |akka.log-dead-letters = $lifecycleEvents
      |akka.log-dead-letters-during-shutdown = $lifecycleEvents
      """.stripMargin))

```

Actually, we do not have any parameter naming "akka.remote.transport-failure-detector.threshold"
see: http://doc.akka.io/docs/akka/2.3.4/general/configuration.html
what we have is "akka.remote.watch-failure-detector.threshold"

Author: CodingCat <zhunansjtu@gmail.com>

Closes #4512 from CodingCat/SPARK-5724 and squashes the following commits:

bafe56e [CodingCat] fix the grammar in configuration doc
338296e [CodingCat] remove failure-detector related info
8bfcfd4 [CodingCat] fix the misconfiguration in AkkaUtils
2015-02-23 11:29:25 +00:00
Ilya Ganelin 95cd643aa9 [SPARK-3885] Provide mechanism to remove accumulators once they are no longer used
Instead of storing a strong reference to accumulators, I've replaced this with a weak reference and updated any code that uses these accumulators to check whether the reference resolves before using the accumulator. A weak reference will be cleared when there is no longer an existing copy of the variable versus using a soft reference in which case accumulators would only be cleared when the GC explicitly ran out of memory.

Author: Ilya Ganelin <ilya.ganelin@capitalone.com>

Closes #4021 from ilganeli/SPARK-3885 and squashes the following commits:

4ba9575 [Ilya Ganelin]  Fixed error in test suite
8510943 [Ilya Ganelin] Extra code
bb76ef0 [Ilya Ganelin] File deleted somehow
283a333 [Ilya Ganelin] Added cleanup method for accumulators to remove stale references within Accumulators.original to accumulators that are now out of scope
345fd4f [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-3885
7485a82 [Ilya Ganelin] Fixed build error
c8e0f2b [Ilya Ganelin] Added working test for accumulator garbage collection
94ce754 [Ilya Ganelin] Still not being properly garbage collected
8722b63 [Ilya Ganelin] Fixing gc test
7414a9c [Ilya Ganelin] Added test for accumulator garbage collection
18d62ec [Ilya Ganelin] Updated to throw Exception when accessing a GCd accumulator
9a81928 [Ilya Ganelin] Reverting permissions changes
28f705c [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-3885
b820ab4b [Ilya Ganelin] reset
d78f4bf [Ilya Ganelin] Removed obsolete comment
0746e61 [Ilya Ganelin] Updated DAGSchedulerSUite to fix bug
3350852 [Ilya Ganelin] Updated DAGScheduler and Suite to correctly use new implementation of WeakRef Accumulator storage
c49066a [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-3885
cbb9023 [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-3885
a77d11b [Ilya Ganelin] Updated Accumulators class to store weak references instead of strong references to allow garbage collection of old accumulators
2015-02-22 22:57:26 -08:00
Aaron Josephs e4f9d03d72 [SPARK-911] allow efficient queries for a range if RDD is partitioned wi...
...th RangePartitioner

Author: Aaron Josephs <ajoseph4@binghamton.edu>

Closes #1381 from aaronjosephs/PLAT-911 and squashes the following commits:

e30ade5 [Aaron Josephs] [SPARK-911] allow efficient queries for a range if RDD is partitioned with RangePartitioner
2015-02-22 22:09:06 -08:00
Evan Yu 7683982faf [SPARK-5860][CORE] JdbcRDD: overflow on large range with high number of partitions
Fix a overflow bug in JdbcRDD when calculating partitions for large BIGINT ids

Author: Evan Yu <ehotou@gmail.com>

Closes #4701 from hotou/SPARK-5860 and squashes the following commits:

9e038d1 [Evan Yu] [SPARK-5860][CORE] Prevent overflowing at the length level
7883ad9 [Evan Yu] [SPARK-5860][CORE] Prevent overflowing at the length level
c88755a [Evan Yu] [SPARK-5860][CORE] switch to BigInt instead of BigDecimal
4e9ff4f [Evan Yu] [SPARK-5860][CORE] JdbcRDD overflow on large range with high number of partitions
2015-02-21 20:40:21 +00:00
Nishkam Ravi d3cbd38c33 SPARK-5841 [CORE] [HOTFIX 2] Memory leak in DiskBlockManager
Continue to see IllegalStateException in YARN cluster mode. Adding a simple workaround for now.

Author: Nishkam Ravi <nravi@cloudera.com>
Author: nishkamravi2 <nishkamravi@gmail.com>
Author: nravi <nravi@c1704.halxg.cloudera.com>

Closes #4690 from nishkamravi2/master_nravi and squashes the following commits:

d453197 [nishkamravi2] Update NewHadoopRDD.scala
6f41a1d [nishkamravi2] Update NewHadoopRDD.scala
0ce2c32 [nishkamravi2] Update HadoopRDD.scala
f7e33c2 [Nishkam Ravi] Merge branch 'master_nravi' of https://github.com/nishkamravi2/spark into master_nravi
ba1eb8b [Nishkam Ravi] Try-catch block around the two occurrences of removeShutDownHook. Deletion of semi-redundant occurrences of expensive operation inShutDown.
71d0e17 [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi
494d8c0 [nishkamravi2] Update DiskBlockManager.scala
3c5ddba [nishkamravi2] Update DiskBlockManager.scala
f0d12de [Nishkam Ravi] Workaround for IllegalStateException caused by recent changes to BlockManager.stop
79ea8b4 [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi
b446edc [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi
5c9a4cb [nishkamravi2] Update TaskSetManagerSuite.scala
535295a [nishkamravi2] Update TaskSetManager.scala
3e1b616 [Nishkam Ravi] Modify test for maxResultSize
9f6583e [Nishkam Ravi] Changes to maxResultSize code (improve error message and add condition to check if maxResultSize > 0)
5f8f9ed [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi
636a9ff [nishkamravi2] Update YarnAllocator.scala
8f76c8b [Nishkam Ravi] Doc change for yarn memory overhead
35daa64 [Nishkam Ravi] Slight change in the doc for yarn memory overhead
5ac2ec1 [Nishkam Ravi] Remove out
dac1047 [Nishkam Ravi] Additional documentation for yarn memory overhead issue
42c2c3d [Nishkam Ravi] Additional changes for yarn memory overhead issue
362da5e [Nishkam Ravi] Additional changes for yarn memory overhead
c726bd9 [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi
f00fa31 [Nishkam Ravi] Improving logging for AM memoryOverhead
1cf2d1e [nishkamravi2] Update YarnAllocator.scala
ebcde10 [Nishkam Ravi] Modify default YARN memory_overhead-- from an additive constant to a multiplier (redone to resolve merge conflicts)
2e69f11 [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi
efd688a [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark
2b630f9 [nravi] Accept memory input as "30g", "512M" instead of an int value, to be consistent with rest of Spark
3bf8fad [nravi] Merge branch 'master' of https://github.com/apache/spark
5423a03 [nravi] Merge branch 'master' of https://github.com/apache/spark
eb663ca [nravi] Merge branch 'master' of https://github.com/apache/spark
df2aeb1 [nravi] Improved fix for ConcurrentModificationIssue (Spark-1097, Hadoop-10456)
6b840f0 [nravi] Undo the fix for SPARK-1758 (the problem is fixed)
5108700 [nravi] Fix in Spark for the Concurrent thread modification issue (SPARK-1097, HADOOP-10456)
681b36f [nravi] Fix for SPARK-1758: failing test org.apache.spark.JavaAPISuite.wholeTextFiles
2015-02-21 09:59:28 -08:00
Sean Owen d3dfebebce SPARK-5744 [CORE] Take 2. RDD.isEmpty / take fails for (empty) RDD of Nothing
Follow-on to https://github.com/apache/spark/pull/4591

Document isEmpty / take / parallelize and their interaction with (an empty) RDD[Nothing] and RDD[Null]. Also, fix a marginally related minor issue with histogram() and EmptyRDD.

CC rxin since you reviewed the last one although I imagine this is an uncontroversial resolution.

Author: Sean Owen <sowen@cloudera.com>

Closes #4698 from srowen/SPARK-5744.2 and squashes the following commits:

9b2a811 [Sean Owen] 2 extra javadoc fixes
d1b9fba [Sean Owen] Document isEmpty / take / parallelize and their interaction with (an empty) RDD[Nothing] and RDD[Null]. Also, fix a marginally related minor issue with histogram() and EmptyRDD.
2015-02-20 10:21:39 +00:00
mcheah 3be92cdac3 [SPARK-4808] Removing minimum number of elements read before spill check
In the general case, Spillable's heuristic of checking for memory stress
on every 32nd item after 1000 items are read is good enough. In general,
we do not want to be enacting the spilling checks until later on in the
job; checking for disk-spilling too early can produce unacceptable
performance impact in trivial cases.

However, there are non-trivial cases, particularly if each serialized
object is large, where checking for the necessity to spill too late
would allow the memory to overflow. Consider if every item is 1.5 MB in
size, and the heap size is 1000 MB. Then clearly if we only try to spill
the in-memory contents to disk after 1000 items are read, we would have
already accumulated 1500 MB of RAM and overflowed the heap.

Patch #3656 attempted to circumvent this by checking the need to spill
on every single item read, but that would cause unacceptable performance
in the general case. However, the convoluted cases above should not be
forced to be refactored to shrink the data items. Therefore it makes
sense that the memory spilling thresholds be configurable.

Author: mcheah <mcheah@palantir.com>

Closes #4420 from mingyukim/memory-spill-configurable and squashes the following commits:

6e2509f [mcheah] [SPARK-4808] Removing minimum number of elements read before spill check
2015-02-19 18:09:22 -08:00
Sean Owen 34b7c35380 SPARK-4682 [CORE] Consolidate various 'Clock' classes
Another one from JoshRosen 's wish list. The first commit is much smaller and removes 2 of the 4 Clock classes. The second is much larger, necessary for consolidating the streaming one. I put together implementations in the way that seemed simplest. Almost all the change is standardizing class and method names.

Author: Sean Owen <sowen@cloudera.com>

Closes #4514 from srowen/SPARK-4682 and squashes the following commits:

5ed3a03 [Sean Owen] Javadoc Clock classes; make ManualClock private[spark]
169dd13 [Sean Owen] Add support for legacy org.apache.spark.streaming clock class names
277785a [Sean Owen] Reduce the net change in this patch by reversing some unnecessary syntax changes along the way
b5e53df [Sean Owen] FakeClock -> ManualClock; getTime() -> getTimeMillis()
160863a [Sean Owen] Consolidate Streaming Clock class into common util Clock
7c956b2 [Sean Owen] Consolidate Clocks except for Streaming Clock
2015-02-19 15:35:23 -08:00
zsxwing 90095bf3ce [SPARK-5423][Core] Cleanup resources in DiskMapIterator.finalize to ensure deleting the temp file
This PR adds a `finalize` method in DiskMapIterator to clean up the resources even if some exception happens during processing data.

Author: zsxwing <zsxwing@gmail.com>

Closes #4219 from zsxwing/SPARK-5423 and squashes the following commits:

d4b2ca6 [zsxwing] Cleanup resources in DiskMapIterator.finalize to ensure deleting the temp file
2015-02-19 18:37:31 +00:00
Andrew Or 38e624a732 [SPARK-5816] Add huge compatibility warning in DriverWrapper
The stability of the new submission gateway assumes that the arguments in `DriverWrapper` are consistent across multiple Spark versions. However, this is not at all clear from the code itself. In fact, this was broken in 20a6013106, which is fortunately OK because both that commit and the original commit that added this gateway are part of the same release.

To prevent this from happening again we should at the very least add a huge warning where appropriate.

Author: Andrew Or <andrew@databricks.com>

Closes #4687 from andrewor14/driver-wrapper-warning and squashes the following commits:

7989b56 [Andrew Or] Add huge compatibility warning
2015-02-19 09:56:25 -08:00
Jacek Lewandowski fb87f44921 SPARK-5548: Fix for AkkaUtilsSuite failure - attempt 2
Author: Jacek Lewandowski <lewandowski.jacek@gmail.com>

Closes #4653 from jacek-lewandowski/SPARK-5548-2-master and squashes the following commits:

0e199b6 [Jacek Lewandowski] SPARK-5548: applied reviewer's comments
843eafb [Jacek Lewandowski] SPARK-5548: Fix for AkkaUtilsSuite failure - attempt 2
2015-02-19 09:53:36 -08:00
Kousuke Saruta 82197ed3bd [SPARK-4949]shutdownCallback in SparkDeploySchedulerBackend should be enclosed by synchronized block.
A variable `shutdownCallback` in SparkDeploySchedulerBackend can be accessed from multiple threads so it should be enclosed by synchronized block.

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

Closes #3781 from sarutak/SPARK-4949 and squashes the following commits:

c146c93 [Kousuke Saruta] Removed "setShutdownCallback" method
c7265dc [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-4949
42ca528 [Kousuke Saruta] Changed the declaration of the variable "shutdownCallback" as a volatile reference instead of AtomicReference
552df7c [Kousuke Saruta] Changed the declaration of the variable "shutdownCallback" as a volatile reference instead of AtomicReference
f556819 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-4949
1b60fd1 [Kousuke Saruta] Improved the locking logics
5942765 [Kousuke Saruta] Enclosed shutdownCallback in SparkDeploySchedulerBackend by synchronized block
2015-02-18 12:20:11 +00:00
Josh Rosen a51fc7ef9a [SPARK-4454] Revert getOrElse() cleanup in DAGScheduler.getCacheLocs()
This method is performance-sensitive and this change wasn't necessary.
2015-02-17 17:45:16 -08:00
Josh Rosen d46d6246d2 [SPARK-4454] Properly synchronize accesses to DAGScheduler cacheLocs map
This patch addresses a race condition in DAGScheduler by properly synchronizing accesses to its `cacheLocs` map.

This map is accessed by the `getCacheLocs` and `clearCacheLocs()` methods, which can be called by separate threads, since DAGScheduler's `getPreferredLocs()` method is called by SparkContext and indirectly calls `getCacheLocs()`.  If this map is cleared by the DAGScheduler event processing thread while a user thread is submitting a job and computing preferred locations, then this can cause the user thread to throw "NoSuchElementException: key not found" errors.

Most accesses to DAGScheduler's internal state do not need synchronization because that state is only accessed from the event processing loop's thread.  An alternative approach to fixing this bug would be to refactor this code so that SparkContext sends the DAGScheduler a message in order to get the list of preferred locations.  However, this would involve more extensive changes to this code and would be significantly harder to backport to maintenance branches since some of the related code has undergone significant refactoring (e.g. the introduction of EventLoop).  Since `cacheLocs` is the only state that's accessed in this way, adding simple synchronization seems like a better short-term fix.

See #3345 for additional context.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #4660 from JoshRosen/SPARK-4454 and squashes the following commits:

12d64ba [Josh Rosen] Properly synchronize accesses to DAGScheduler cacheLocs map.
2015-02-17 17:39:58 -08:00
Burak Yavuz ae6cfb3acd [SPARK-5811] Added documentation for maven coordinates and added Spark Packages support
Documentation for maven coordinates + Spark Package support. Added pyspark tests for `--packages`

Author: Burak Yavuz <brkyvz@gmail.com>
Author: Davies Liu <davies@databricks.com>

Closes #4662 from brkyvz/SPARK-5811 and squashes the following commits:

56ccccd [Burak Yavuz] fixed broken test
64cb8ee [Burak Yavuz] passed pep8 on local
c07b81e [Burak Yavuz] fixed pep8
a8bd6b7 [Burak Yavuz] submit PR
4ef4046 [Burak Yavuz] ready for PR
8fb02e5 [Burak Yavuz] merged master
25c9b9f [Burak Yavuz] Merge branch 'master' of github.com:apache/spark into python-jar
560d13b [Burak Yavuz] before PR
17d3f76 [Davies Liu] support .jar as python package
a3eb717 [Burak Yavuz] Merge branch 'master' of github.com:apache/spark into SPARK-5811
c60156d [Burak Yavuz] [SPARK-5811] Added documentation for maven coordinates
2015-02-17 17:23:22 -08:00
Davies Liu c3d2b90bde [SPARK-5785] [PySpark] narrow dependency for cogroup/join in PySpark
Currently, PySpark does not support narrow dependency during cogroup/join when the two RDDs have the partitioner, another unnecessary shuffle stage will come in.

The Python implementation of cogroup/join is different than Scala one, it depends on union() and partitionBy(). This patch will try to use PartitionerAwareUnionRDD() in union(), when all the RDDs have the same partitioner. It also fix `reservePartitioner` in all the map() or mapPartitions(), then partitionBy() can skip the unnecessary shuffle stage.

Author: Davies Liu <davies@databricks.com>

Closes #4629 from davies/narrow and squashes the following commits:

dffe34e [Davies Liu] improve test, check number of stages for join/cogroup
1ed3ba2 [Davies Liu] Merge branch 'master' of github.com:apache/spark into narrow
4d29932 [Davies Liu] address comment
cc28d97 [Davies Liu] add unit tests
940245e [Davies Liu] address comments
ff5a0a6 [Davies Liu] skip the partitionBy() on Python side
eb26c62 [Davies Liu] narrow dependency in PySpark
2015-02-17 16:54:57 -08:00
Davies Liu 445a755b88 [SPARK-4172] [PySpark] Progress API in Python
This patch bring the pull based progress API into Python, also a example in Python.

Author: Davies Liu <davies@databricks.com>

Closes #3027 from davies/progress_api and squashes the following commits:

b1ba984 [Davies Liu] fix style
d3b9253 [Davies Liu] add tests, mute the exception after stop
4297327 [Davies Liu] Merge branch 'master' of github.com:apache/spark into progress_api
969fa9d [Davies Liu] Merge branch 'master' of github.com:apache/spark into progress_api
25590c9 [Davies Liu] update with Java API
360de2d [Davies Liu] Merge branch 'master' of github.com:apache/spark into progress_api
c0f1021 [Davies Liu] Merge branch 'master' of github.com:apache/spark into progress_api
023afb3 [Davies Liu] add Python API and example for progress API
2015-02-17 13:36:43 -08:00
Sean Owen 49c19fdbad SPARK-5841 [CORE] [HOTFIX] Memory leak in DiskBlockManager
Avoid call to remove shutdown hook being called from shutdown hook

CC pwendell JoshRosen MattWhelan

Author: Sean Owen <sowen@cloudera.com>

Closes #4648 from srowen/SPARK-5841.2 and squashes the following commits:

51548db [Sean Owen] Avoid call to remove shutdown hook being called from shutdown hook
2015-02-17 19:40:06 +00:00
xukun 00228947 b271c265b7 [SPARK-5661]function hasShutdownDeleteTachyonDir should use shutdownDeleteTachyonPaths to determine whether contains file
hasShutdownDeleteTachyonDir(file: TachyonFile) should use shutdownDeleteTachyonPaths(not shutdownDeletePaths) to determine Whether contain file. To solve it ,delete two unused function.

Author: xukun 00228947 <xukun.xu@huawei.com>
Author: viper-kun <xukun.xu@huawei.com>

Closes #4418 from viper-kun/deleteunusedfun and squashes the following commits:

87340eb [viper-kun] fix style
3d6c69e [xukun 00228947] fix bug
2bc397e [xukun 00228947] deleteunusedfun
2015-02-17 18:59:41 +00:00
Ryan Williams d8f69cf788 [SPARK-5778] throw if nonexistent metrics config file provided
previous behavior was to log an error; this is fine in the general
case where no `spark.metrics.conf` parameter was specified, in which
case a default `metrics.properties` is looked for, and the execption
logged and suppressed if it doesn't exist.

if the user has purposefully specified a metrics.conf file, however,
it makes more sense to show them an error when said file doesn't
exist.

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

Closes #4571 from ryan-williams/metrics and squashes the following commits:

5bccb14 [Ryan Williams] private-ize some MetricsConfig members
08ff998 [Ryan Williams] rename METRICS_CONF: DEFAULT_METRICS_CONF_FILENAME
f4d7fab [Ryan Williams] fix tests
ad24b0e [Ryan Williams] add "metrics.properties" to .rat-excludes
94e810b [Ryan Williams] throw if nonexistent Sink class is specified
31d2c30 [Ryan Williams] metrics code review feedback
56287db [Ryan Williams] throw if nonexistent metrics config file provided
2015-02-17 10:57:16 -08:00
Josh Rosen ee6e3eff02 Revert "[SPARK-5363] [PySpark] check ending mark in non-block way"
This reverts commits ac6fe67e1d and c06e42f2c1.
2015-02-17 07:49:02 -08:00
Patrick Wendell c06e42f2c1 HOTFIX: Style issue causing build break
Caused by #4601
2015-02-16 22:11:28 -08:00
Davies Liu ac6fe67e1d [SPARK-5363] [PySpark] check ending mark in non-block way
There is chance of dead lock that the Python process is waiting for ending mark from JVM, but which is eaten by corrupted stream.

This PR checks the ending mark from Python in non-block way, so it will not blocked by Python process.

There is a small chance that the ending mark is sent by Python process but not available right now, then Python process will not be used.

cc JoshRosen pwendell

Author: Davies Liu <davies@databricks.com>

Closes #4601 from davies/freeze and squashes the following commits:

e15a8c3 [Davies Liu] update logging
890329c [Davies Liu] Merge branch 'freeze' of github.com:davies/spark into freeze
2bd2228 [Davies Liu] add more logging
656d544 [Davies Liu] Update PythonRDD.scala
05e1085 [Davies Liu] check ending mark in non-block way
2015-02-16 20:32:03 -08:00
Josh Rosen 58a82a7882 [SPARK-5849] Handle more types of invalid JSON requests in SubmitRestProtocolMessage.parseAction
This patch improves SubmitRestProtocol's handling of invalid JSON requests in cases where those requests were parsable as JSON but not as JSON objects (e.g. they could be parsed as ararys or strings).  I replaced an unchecked cast with pattern-matching and added a new test case.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #4637 from JoshRosen/rest-protocol-cast and squashes the following commits:

b3f282b [Josh Rosen] [SPARK-5849] Handle more types of invalid JSON in SubmitRestProtocolMessage.parseAction
2015-02-16 18:08:02 -08:00
Davies Liu b1bd1dd322 [SPARK-5788] [PySpark] capture the exception in python write thread
The exception in Python writer thread will shutdown executor.

Author: Davies Liu <davies@databricks.com>

Closes #4577 from davies/exception and squashes the following commits:

eb0ceff [Davies Liu] Update PythonRDD.scala
139b0db [Davies Liu] capture the exception in python write thread
2015-02-16 17:57:14 -08:00
Matt Whelan 1294a6e01a SPARK-5848: tear down the ConsoleProgressBar timer
The timer is a GC root, and failing to terminate it leaks SparkContext
instances.

Author: Matt Whelan <mwhelan@perka.com>

Closes #4635 from MattWhelan/SPARK-5848 and squashes the following commits:

2a1e8a5 [Matt Whelan] SPARK-5848: teardown the ConsoleProgressBar timer
2015-02-17 00:59:49 +00:00
Josh Rosen 0cfda8461f [SPARK-2313] Use socket to communicate GatewayServer port back to Python driver
This patch changes PySpark so that the GatewayServer's port is communicated back to the Python process that launches it over a local socket instead of a pipe.  The old pipe-based approach was brittle and could fail if `spark-submit` printed unexpected to stdout.

To accomplish this, I wrote a custom `PythonGatewayServer.main()` function to use in place of Py4J's `GatewayServer.main()`.

Closes #3424.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #4603 from JoshRosen/SPARK-2313 and squashes the following commits:

6a7740b [Josh Rosen] Remove EchoOutputThread since it's no longer needed
0db501f [Josh Rosen] Use select() so that we don't block if GatewayServer dies.
9bdb4b6 [Josh Rosen] Handle case where getListeningPort returns -1
3fb7ed1 [Josh Rosen] Remove stdout=PIPE
2458934 [Josh Rosen] Use underscore to mark env var. as private
d12c95d [Josh Rosen] Use Logging and Utils.tryOrExit()
e5f9730 [Josh Rosen] Wrap everything in a giant try-block
2f70689 [Josh Rosen] Use stdin PIPE to share fate with driver
8bf956e [Josh Rosen] Initial cut at passing Py4J gateway port back to driver via socket
2015-02-16 15:25:11 -08:00
Matt Whelan bb05982dd2 SPARK-5841: remove DiskBlockManager shutdown hook on stop
After a call to stop, the shutdown hook is redundant, and causes a
memory leak.

Author: Matt Whelan <mwhelan@perka.com>

Closes #4627 from MattWhelan/SPARK-5841 and squashes the following commits:

d5f5c7f [Matt Whelan] SPARK-5841: remove DiskBlockManager shutdown hook on stop
2015-02-16 22:54:32 +00:00
Josh Rosen d06d5ee9b3 [SPARK-5227] [SPARK-5679] Disable FileSystem cache in WholeTextFileRecordReaderSuite
This patch fixes two difficult-to-reproduce Jenkins test failures in InputOutputMetricsSuite (SPARK-5227 and SPARK-5679).  The problem was that WholeTextFileRecordReaderSuite modifies the `fs.local.block.size` Hadoop configuration and this change was affecting subsequent test suites due to Hadoop's caching of FileSystem instances (see HADOOP-8490 for more details).

The fix implemented here is to disable FileSystem caching in WholeTextFileRecordReaderSuite.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #4599 from JoshRosen/inputoutputsuite-fix and squashes the following commits:

47dc447 [Josh Rosen] [SPARK-5227] [SPARK-5679] Disable FileSystem cache in WholeTextFileRecordReaderSuite
2015-02-13 17:45:31 -08:00
Andrew Or 5d3cc6b3d7 [HOTFIX] Fix build break in MesosSchedulerBackendSuite 2015-02-13 13:10:29 -08:00
Josh Rosen 077eec2d9d [SPARK-5735] Replace uses of EasyMock with Mockito
This patch replaces all uses of EasyMock with Mockito.  There are two motivations for this:

1. We should use a single mocking framework in our tests in order to keep things consistent.
2. EasyMock may be responsible for non-deterministic unit test failures due to its Objensis dependency (see SPARK-5626 for more details).

Most of these changes are fairly mechanical translations of Mockito code to EasyMock, although I made a small change that strengthens the assertions in one test in KinesisReceiverSuite.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #4578 from JoshRosen/SPARK-5735-remove-easymock and squashes the following commits:

0ab192b [Josh Rosen] Import sorting plus two minor changes to more closely match old semantics.
977565b [Josh Rosen] Remove EasyMock from build.
fae1d8f [Josh Rosen] Remove EasyMock usage in KinesisReceiverSuite.
7cca486 [Josh Rosen] Remove EasyMock usage in MesosSchedulerBackendSuite
fc5e94d [Josh Rosen] Remove EasyMock in CacheManagerSuite
2015-02-13 09:55:36 -08:00
Ryan Williams fc6d3e796a [SPARK-5783] Better eventlog-parsing error messages
Author: Ryan Williams <ryan.blake.williams@gmail.com>

Closes #4573 from ryan-williams/history and squashes the following commits:

a8647ec [Ryan Williams] fix test calls to .replay()
98aa3fe [Ryan Williams] include filename in history-parsing error message
8deecf0 [Ryan Williams] add line number to history-parsing error message
b668b52 [Ryan Williams] add log info line to history-eventlog parsing
2015-02-13 09:47:26 -08:00
uncleGen c0ccd25641 [SPARK-5732][CORE]:Add an option to print the spark version in spark script.
Naturally, we may need to add an option to print the spark version in spark script. It is pretty common in script tool.
![9](https://cloud.githubusercontent.com/assets/7402327/6183331/cab1b74e-b38e-11e4-9daa-e26e6015cff3.JPG)

Author: uncleGen <hustyugm@gmail.com>
Author: genmao.ygm <genmao.ygm@alibaba-inc.com>

Closes #4522 from uncleGen/master-clean-150211 and squashes the following commits:

9f2127c [genmao.ygm] revert the behavior of "-v"
015ddee [uncleGen] minor changes
463f02c [uncleGen] minor changes
2015-02-13 09:44:13 -08:00
Andrew Or 1d5663e92c [SPARK-5760][SPARK-5761] Fix standalone rest protocol corner cases + revamp tests
The changes are summarized in the commit message. Test or test-related code accounts for 90% of the lines changed.

Author: Andrew Or <andrew@databricks.com>

Closes #4557 from andrewor14/rest-tests and squashes the following commits:

b4dc980 [Andrew Or] Merge branch 'master' of github.com:apache/spark into rest-tests
b55e40f [Andrew Or] Add test for unknown fields
cc96993 [Andrew Or] private[spark] -> private[rest]
578cf45 [Andrew Or] Clean up test code a little
d82d971 [Andrew Or] v1 -> serverVersion
ea48f65 [Andrew Or] Merge branch 'master' of github.com:apache/spark into rest-tests
00999a8 [Andrew Or] Revamp tests + fix a few corner cases
2015-02-12 14:47:52 -08:00
Kay Ousterhout 47c73d410a [SPARK-5762] Fix shuffle write time for sort-based shuffle
mateiz was excluding the time to write this final file from the shuffle write time intentional?

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #4559 from kayousterhout/SPARK-5762 and squashes the following commits:

5c6f3d9 [Kay Ousterhout] Use foreach
94e4237 [Kay Ousterhout] Removed open time metrics added inadvertently
ace156c [Kay Ousterhout] Moved metrics to finally block
d773276 [Kay Ousterhout] Use nano time
5a59906 [Kay Ousterhout] [SPARK-5762] Fix shuffle write time for sort-based shuffle
2015-02-12 14:46:37 -08:00
Kay Ousterhout 893d6fd704 [SPARK-5645] Added local read bytes/time to task metrics
ksakellis I stumbled on your JIRA for this yesterday; I know it's assigned to you but I'd already done this for my own uses a while ago so thought I could help save you the work of doing it!  Hopefully this doesn't duplicate any work you've already done.

Here's a screenshot of what the UI looks like:
![image](https://cloud.githubusercontent.com/assets/1108612/6135352/c03e7276-b11c-11e4-8f11-c6aefe1f35b9.png)
Based on a discussion with pwendell, I put the data read remotely in as an additional metric rather than showing it in brackets as you'd suggested, Kostas.  The assumption here is that the average user doesn't care about the differentiation between local / remote data, so it's better not to pollute the UI.

I also added data about the local read time, which I've found very helpful for debugging, but I didn't put it in the UI because I think it's probably something not a ton of people will need to use.

With this change, the total read time and total write time shown in the UI will be equal, fixing a long-term source of user confusion:
![image](https://cloud.githubusercontent.com/assets/1108612/6135399/25f14490-b11d-11e4-8086-20be5f4002e6.png)

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #4510 from kayousterhout/SPARK-5645 and squashes the following commits:

4a0182c [Kay Ousterhout] oops
5f5da1b [Kay Ousterhout] Small style fix
5da04cf [Kay Ousterhout] Addressed more comments from Kostas
ba05149 [Kay Ousterhout] Remove parens
a9dc685 [Kay Ousterhout] Kostas comment, test fix
33d2e2d [Kay Ousterhout] Merge remote-tracking branch 'upstream/master' into SPARK-5645
347e2cd [Kay Ousterhout] [SPARK-5645] Added local read bytes/time to task metrics
2015-02-12 14:36:27 -08:00
Andrew Rowson 466b1f671b [SPARK-5655] Don't chmod700 application files if running in YARN
[Was previously PR4507]

As per SPARK-5655, recently committed code chmod 700s all application files created on the local fs by a spark executor. This is both unnecessary and broken on YARN, where files created in the nodemanager's working directory are already owned by the user running the job and the 'yarn' group. Group read permission is also needed for the auxiliary shuffle service to be able to read the files, as this is running as the 'yarn' user.

Author: Andrew Rowson <github@growse.com>

Closes #4509 from growse/master and squashes the following commits:

7ca993c [Andrew Rowson] Moved chmod700 functionality into Utils.getOrCreateLocalRootDirs
f57ce6b [Andrew Rowson] [SPARK-5655] Don't chmod700 application files if running in a YARN container
2015-02-12 18:41:39 +00:00
guliangliang 1ac099e3e0 [SPARK-5733] Error Link in Pagination of HistroyPage when showing Incomplete Applications
The links in pagination of HistroyPage is wrong when showing Incomplete Applications.

If "2" is click on the following page "http://history-server:18080/?page=1&showIncomplete=true", it will go to "http://history-server:18080/?page=2" instead of "http://history-server:18080/?page=2&showIncomplete=true".

Author: guliangliang <guliangliang@qiyi.com>

Closes #4523 from marsishandsome/Spark5733 and squashes the following commits:

9d7b593 [guliangliang] [SPARK-5733] Error Link in Pagination of HistroyPage when showing Incomplete Applications
2015-02-11 15:55:49 +00:00
Andrew Or b969182659 [SPARK-5729] Potential NPE in standalone REST API
If the user specifies a bad REST URL, the server will throw an NPE instead of propagating the error back. This is because the default `ErrorServlet` has the wrong prefix. This is a one line fix. I am will add more comprehensive tests in a separate patch.

Author: Andrew Or <andrew@databricks.com>

Closes #4518 from andrewor14/rest-npe and squashes the following commits:

16b15bc [Andrew Or] Correct ErrorServlet context prefix
2015-02-10 20:19:14 -08:00
mcheah 1cb3770075 [SPARK-4879] Use driver to coordinate Hadoop output committing for speculative tasks
Previously, SparkHadoopWriter always committed its tasks without question. The problem is that when speculation is enabled sometimes this can result in multiple tasks committing their output to the same file. Even though an HDFS-writing task may be re-launched due to speculation, the original task is not killed and may eventually commit as well.

This can cause strange race conditions where multiple tasks that commit interfere with each other, with the result being that some partition files are actually lost entirely. For more context on these kinds of scenarios, see SPARK-4879.

In Hadoop MapReduce jobs, the application master is a central coordinator that authorizes whether or not any given task can commit. Before a task commits its output, it queries the application master as to whether or not such a commit is safe, and the application master does bookkeeping as tasks are requesting commits. Duplicate tasks that would write to files that were already written to from other tasks are prohibited from committing.

This patch emulates that functionality - the crucial missing component was a central arbitrator, which is now a module called the OutputCommitCoordinator. The coordinator lives on the driver and the executors can obtain a reference to this actor and request its permission to commit. As tasks commit and are reported as completed successfully or unsuccessfully by the DAGScheduler, the commit coordinator is informed of the task completion events as well to update its internal state.

Future work includes more rigorous unit testing and extra optimizations should this patch cause a performance regression. It is unclear what the overall cost of communicating back to the driver on every hadoop-committing task will be. It's also important for those hitting this issue to backport this onto previous version of Spark because the bug has serious consequences, that is, data is lost.

Currently, the OutputCommitCoordinator is only used when `spark.speculation` is true.  It can be disabled by setting `spark.hadoop.outputCommitCoordination.enabled=false` in SparkConf.

This patch is an updated version of #4155 (by mccheah), which in turn was an updated version of this PR.

Closes #4155.

Author: mcheah <mcheah@palantir.com>
Author: Josh Rosen <joshrosen@databricks.com>

Closes #4066 from JoshRosen/SPARK-4879-sparkhadoopwriter-fix and squashes the following commits:

658116b [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4879-sparkhadoopwriter-fix
ed783b2 [Josh Rosen] Address Andrew’s feedback.
e7be65a [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4879-sparkhadoopwriter-fix
14861ea [Josh Rosen] splitID -> partitionID in a few places
ed8b554 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4879-sparkhadoopwriter-fix
48d5c1c [Josh Rosen] Roll back copiesRunning change in TaskSetManager
3969f5f [Josh Rosen] Re-enable guarding of commit coordination with spark.speculation setting.
ede7590 [Josh Rosen] Add test to ensure that a job that denies all commits cannot complete successfully.
97da5fe [Josh Rosen] Use actor only for RPC; call methods directly in DAGScheduler.
f582574 [Josh Rosen] Some cleanup in OutputCommitCoordinatorSuite
a7c0e29 [Josh Rosen] Create fake TaskInfo using dummy fields instead of Mockito.
997b41b [Josh Rosen] Roll back unnecessary DAGSchedulerSingleThreadedProcessLoop refactoring:
459310a [Josh Rosen] Roll back TaskSetManager changes that broke other tests.
dd00b7c [Josh Rosen] Move CommitDeniedException to executors package; remove `@DeveloperAPI` annotation.
c79df98 [Josh Rosen] Some misc. code style + doc changes:
f7d69c5 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4879-sparkhadoopwriter-fix
92e6dc9 [Josh Rosen] Bug fix: use task ID instead of StageID to index into authorizedCommitters.
b344bad [Josh Rosen] (Temporarily) re-enable “always coordinate” for testing purposes.
0aec91e [Josh Rosen] Only coordinate when speculation is enabled; add configuration option to bypass new coordination.
594e41a [mcheah] Fixing a scalastyle error
60a47f4 [mcheah] Writing proper unit test for OutputCommitCoordinator and fixing bugs.
d63f63f [mcheah] Fixing compiler error
9fe6495 [mcheah] Fixing scalastyle
1df2a91 [mcheah] Throwing exception if SparkHadoopWriter commit denied
d431144 [mcheah] Using more concurrency to process OutputCommitCoordinator requests.
c334255 [mcheah] Properly handling messages that could be sent after actor shutdown.
8d5a091 [mcheah] Was mistakenly serializing the accumulator in test suite.
9c6a4fa [mcheah] More OutputCommitCoordinator cleanup on stop()
78eb1b5 [mcheah] Better OutputCommitCoordinatorActor stopping; simpler canCommit
83de900 [mcheah] Making the OutputCommitCoordinatorMessage serializable
abc7db4 [mcheah] TaskInfo can't be null in DAGSchedulerSuite
f135a8e [mcheah] Moving the output commit coordinator from class into method.
1c2b219 [mcheah] Renaming oudated names for test function classes
66a71cd [mcheah] Removing whitespace modifications
6b543ba [mcheah] Removing redundant accumulator in unit test
c9decc6 [mcheah] Scalastyle fixes
bc80770 [mcheah] Unit tests for OutputCommitCoordinator
6e6f748 [mcheah] [SPARK-4879] Use the Spark driver to authorize Hadoop commits.
2015-02-10 20:12:18 -08:00
Marcelo Vanzin ed167e70c6 [SPARK-5493] [core] Add option to impersonate user.
Hadoop has a feature that allows users to impersonate other users
when submitting applications or talking to HDFS, for example. These
impersonated users are referred generally as "proxy users".

Services such as Oozie or Hive use this feature to run applications
as the requesting user.

This change makes SparkSubmit accept a new command line option to
run the application as a proxy user. It also fixes the plumbing
of the user name through the UI (and a couple of other places) to
refer to the correct user running the application, which can be
different than `sys.props("user.name")` even without proxies (e.g.
when using kerberos).

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #4405 from vanzin/SPARK-5493 and squashes the following commits:

df82427 [Marcelo Vanzin] Clarify the reason for the special exception handling.
05bfc08 [Marcelo Vanzin] Remove unneeded annotation.
4840de9 [Marcelo Vanzin] Review feedback.
8af06ff [Marcelo Vanzin] Fix usage string.
2e4fa8f [Marcelo Vanzin] Merge branch 'master' into SPARK-5493
b6c947d [Marcelo Vanzin] Merge branch 'master' into SPARK-5493
0540d38 [Marcelo Vanzin] [SPARK-5493] [core] Add option to impersonate user.
2015-02-10 17:19:10 -08:00
Sephiroth-Lin 52983d7f4f [SPARK-5644] [Core]Delete tmp dir when sc is stop
When we run driver as a service, and for each time we run job we only call sc.stop, then will not delete tmp dir create by HttpFileServer and SparkEnv, it will be deleted until the service process exit, so we need to delete these tmp dirs when sc is stop directly.

Author: Sephiroth-Lin <linwzhong@gmail.com>

Closes #4412 from Sephiroth-Lin/bug-fix-master-01 and squashes the following commits:

fbbc785 [Sephiroth-Lin] using an interpolated string
b968e14 [Sephiroth-Lin] using an interpolated string
4edf394 [Sephiroth-Lin] rename the variable and update comment
1339c96 [Sephiroth-Lin] add a member to store the reference of tmp dir
b2018a5 [Sephiroth-Lin] check sparkFilesDir before delete
f48a3c6 [Sephiroth-Lin] don't check sparkFilesDir, check executorId
dd9686e [Sephiroth-Lin] format code
b38e0f0 [Sephiroth-Lin] add dir check before delete
d7ccc64 [Sephiroth-Lin] Change log level
1d70926 [Sephiroth-Lin] update comment
e2a2b1b [Sephiroth-Lin] update comment
aeac518 [Sephiroth-Lin] Delete tmp dir when sc is stop
c0d5b28 [Sephiroth-Lin] Delete tmp dir when sc is stop
2015-02-10 23:23:35 +00:00
Andrew Or b640c841fc [HOTFIX][SPARK-4136] Fix compilation and tests 2015-02-10 11:18:01 -08:00
Sandy Ryza 69bc3bb6cf SPARK-4136. Under dynamic allocation, cancel outstanding executor requests when no longer needed
This takes advantage of the changes made in SPARK-4337 to cancel pending requests to YARN when they are no longer needed.

Each time the timer in `ExecutorAllocationManager` strikes, we compute `maxNumNeededExecutors`, the maximum number of executors we could fill with the current load.  This is calculated as the total number of running and pending tasks divided by the number of cores per executor.  If `maxNumNeededExecutors` is below the total number of running and pending executors, we call `requestTotalExecutors(maxNumNeededExecutors)` to let the cluster manager know that it should cancel any pending requests above this amount.  If not, `maxNumNeededExecutors` is just used as a bound in alongside the configured `maxExecutors` to limit the number of new requests.

The patch modifies the API exposed by `ExecutorAllocationClient` for requesting additional executors by moving from `requestExecutors` to `requestTotalExecutors`.  This makes the communication between the `ExecutorAllocationManager` and the `YarnAllocator` easier to reason about and removes some state that needed to be kept in the `CoarseGrainedSchedulerBackend`.  I think an argument can be made that this makes for a less attractive user-facing API in `SparkContext`, but I'm having trouble envisioning situations where a user would want to use either of these APIs.

This will likely break some tests, but I wanted to get feedback on the approach before adding tests and polishing.

Author: Sandy Ryza <sandy@cloudera.com>

Closes #4168 from sryza/sandy-spark-4136 and squashes the following commits:

37ce77d [Sandy Ryza] Warn on negative number
cd3b2ff [Sandy Ryza] SPARK-4136
2015-02-10 11:12:06 -08:00
Sean Owen 2d1e916730 SPARK-5239 [CORE] JdbcRDD throws "java.lang.AbstractMethodError: oracle.jdbc.driver.xxxxxx.isClosed()Z"
This is a completion of https://github.com/apache/spark/pull/4033 which was withdrawn for some reason.

Author: Sean Owen <sowen@cloudera.com>

Closes #4470 from srowen/SPARK-5239.2 and squashes the following commits:

2398bde [Sean Owen] Avoid use of JDBC4-only isClosed()
2015-02-10 09:19:01 +00:00
Kay Ousterhout a2d33d0b01 [SPARK-5701] Only set ShuffleReadMetrics when task has shuffle deps
The updateShuffleReadMetrics method in TaskMetrics (called by the executor heartbeater) will currently always add a ShuffleReadMetrics to TaskMetrics (with values set to 0), even when the task didn't read any shuffle data. ShuffleReadMetrics should only be added if the task reads shuffle data.

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #4488 from kayousterhout/SPARK-5701 and squashes the following commits:

673ed58 [Kay Ousterhout] SPARK-5701: Only set ShuffleReadMetrics when task has shuffle deps
2015-02-09 21:22:09 -08:00
Andrew Or a95ed52157 [SPARK-5703] AllJobsPage throws empty.max exception
If you have a `SparkListenerJobEnd` event without the corresponding `SparkListenerJobStart` event, then `JobProgressListener` will create an empty `JobUIData` with an empty `stageIds` list. However, later in `AllJobsPage` we call `stageIds.max`. If this is empty, it will throw an exception.

This crashed my history server.

Author: Andrew Or <andrew@databricks.com>

Closes #4490 from andrewor14/jobs-page-max and squashes the following commits:

21797d3 [Andrew Or] Check nonEmpty before calling max
2015-02-09 21:18:48 -08:00
Marcelo Vanzin 20a6013106 [SPARK-2996] Implement userClassPathFirst for driver, yarn.
Yarn's config option `spark.yarn.user.classpath.first` does not work the same way as
`spark.files.userClassPathFirst`; Yarn's version is a lot more dangerous, in that it
modifies the system classpath, instead of restricting the changes to the user's class
loader. So this change implements the behavior of the latter for Yarn, and deprecates
the more dangerous choice.

To be able to achieve feature-parity, I also implemented the option for drivers (the existing
option only applies to executors). So now there are two options, each controlling whether
to apply userClassPathFirst to the driver or executors. The old option was deprecated, and
aliased to the new one (`spark.executor.userClassPathFirst`).

The existing "child-first" class loader also had to be fixed. It didn't handle resources, and it
was also doing some things that ended up causing JVM errors depending on how things
were being called.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #3233 from vanzin/SPARK-2996 and squashes the following commits:

9cf9cf1 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
a1499e2 [Marcelo Vanzin] Remove SPARK_HOME propagation.
fa7df88 [Marcelo Vanzin] Remove 'test.resource' file, create it dynamically.
a8c69f1 [Marcelo Vanzin] Review feedback.
cabf962 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
a1b8d7e [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
3f768e3 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
2ce3c7a [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
0e6d6be [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
70d4044 [Marcelo Vanzin] Fix pyspark/yarn-cluster test.
0fe7777 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
0e6ef19 [Marcelo Vanzin] Move class loaders around and make names more meaninful.
fe970a7 [Marcelo Vanzin] Review feedback.
25d4fed [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
3cb6498 [Marcelo Vanzin] Call the right loadClass() method on the parent.
fbb8ab5 [Marcelo Vanzin] Add locking in loadClass() to avoid deadlocks.
2e6c4b7 [Marcelo Vanzin] Mention new setting in documentation.
b6497f9 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
a10f379 [Marcelo Vanzin] Some feedback.
3730151 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
f513871 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
44010b6 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
7b57cba [Marcelo Vanzin] Remove now outdated message.
5304d64 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
35949c8 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
54e1a98 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
d1273b2 [Marcelo Vanzin] Add test file to rat exclude.
fa1aafa [Marcelo Vanzin] Remove write check on user jars.
89d8072 [Marcelo Vanzin] Cleanups.
a963ea3 [Marcelo Vanzin] Implement spark.driver.userClassPathFirst for standalone cluster mode.
50afa5f [Marcelo Vanzin] Fix Yarn executor command line.
7d14397 [Marcelo Vanzin] Register user jars in executor up front.
7f8603c [Marcelo Vanzin] Fix yarn-cluster mode without userClassPathFirst.
20373f5 [Marcelo Vanzin] Fix ClientBaseSuite.
55c88fa [Marcelo Vanzin] Run all Yarn integration tests via spark-submit.
0b64d92 [Marcelo Vanzin] Add deprecation warning to yarn option.
4a84d87 [Marcelo Vanzin] Fix the child-first class loader.
d0394b8 [Marcelo Vanzin] Add "deprecated configs" to SparkConf.
46d8cf2 [Marcelo Vanzin] Update doc with new option, change name to "userClassPathFirst".
a314f2d [Marcelo Vanzin] Enable driver class path isolation in SparkSubmit.
91f7e54 [Marcelo Vanzin] [yarn] Enable executor class path isolation.
a853e74 [Marcelo Vanzin] Re-work CoarseGrainedExecutorBackend command line arguments.
89522ef [Marcelo Vanzin] Add class path isolation support for Yarn cluster mode.
2015-02-09 21:17:28 -08:00
KaiXinXiaoLei 31d435ecfd Add a config option to print DAG.
Add a config option "spark.rddDebug.enable" to check whether to print DAG info. When "spark.rddDebug.enable" is true, it will print information about DAG in the log.

Author: KaiXinXiaoLei <huleilei1@huawei.com>

Closes #4257 from KaiXinXiaoLei/DAGprint and squashes the following commits:

d9fe42e [KaiXinXiaoLei] change  log info
c27ee76 [KaiXinXiaoLei] change log info
83c2b32 [KaiXinXiaoLei] change config option
adcb14f [KaiXinXiaoLei] change the file.
f4e7b9e [KaiXinXiaoLei] add a option to print DAG
2015-02-09 20:58:58 -08:00
Andrew Or d302c4800b [SPARK-5698] Do not let user request negative # of executors
Otherwise we might crash the ApplicationMaster. Why? Please see https://issues.apache.org/jira/browse/SPARK-5698.

sryza I believe this is also relevant in your patch #4168.

Author: Andrew Or <andrew@databricks.com>

Closes #4483 from andrewor14/da-negative and squashes the following commits:

53ed955 [Andrew Or] Throw IllegalArgumentException instead
0e89fd5 [Andrew Or] Check against negative requests
2015-02-09 17:33:29 -08:00
mcheah 6fe70d8432 [SPARK-5691] Fixing wrong data structure lookup for dupe app registratio...
In Master's registerApplication method, it checks if the application had
already registered by examining the addressToWorker hash map. In reality,
it should refer to the addressToApp data structure, as this is what
really tracks which apps have been registered.

Author: mcheah <mcheah@palantir.com>

Closes #4477 from mccheah/spark-5691 and squashes the following commits:

efdc573 [mcheah] [SPARK-5691] Fixing wrong data structure lookup for dupe app registration
2015-02-09 13:20:14 -08:00
Kirill A. Korinskiy 23a99dabf1 [SPARK-5672][Web UI] Don't return ERROR 500 when have missing args
Spark web UI return `HTTP ERROR 500` when GET arguments is missing.

Author: Kirill A. Korinskiy <catap@catap.ru>

Closes #4239 from catap/ui_500 and squashes the following commits:

520e180 [Kirill A. Korinskiy] [SPARK-5672][Web UI] Return `HTTP ERROR 400` when have missing args
2015-02-08 10:31:46 +00:00
Jacek Lewandowski dd4cb33a27 SPARK-5408: Use -XX:MaxPermSize specified by user instead of default in ...
...ExecutorRunner and DriverRunner

Author: Jacek Lewandowski <lewandowski.jacek@gmail.com>

Closes #4203 from jacek-lewandowski/SPARK-5408-1.3 and squashes the following commits:

d913686 [Jacek Lewandowski] SPARK-5408: Use -XX:MaxPermSize specified by used instead of default in ExecutorRunner and DriverRunner
2015-02-07 15:58:04 +00:00
Andrew Or 1390e56fa8 [SPARK-5388] Provide a stable application submission gateway for standalone cluster mode
The goal is to provide a stable, REST-based application submission gateway that is not inherently based on Akka, which is unstable across versions. This PR targets standalone cluster mode, but is implemented in a general enough manner that can be potentially extended to other modes in the future. Client mode is currently not included in the changes here because there are many more Akka messages exchanged there.

As of the changes here, the Master will advertise two ports, 7077 and 6066. We need to keep around the old one (7077) for client mode and older versions of Spark submit. However, all new versions of Spark submit will use the REST gateway (6066).

By the way this includes ~700 lines of tests and ~200 lines of license.

Author: Andrew Or <andrew@databricks.com>

Closes #4216 from andrewor14/rest and squashes the following commits:

8d7ce07 [Andrew Or] Merge branch 'master' of github.com:apache/spark into rest
6f0c597 [Andrew Or] Use nullable fields for integer and boolean values
dfe4bd7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into rest
b9e2a08 [Andrew Or] Minor comments
02b5cea [Andrew Or] Fix tests
d2b1ef8 [Andrew Or] Comment changes + minor code refactoring across the board
9c82a36 [Andrew Or] Minor comment and wording updates
b4695e7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into rest
c9a8ad7 [Andrew Or] Do not include appResource and mainClass as properties
6fc7670 [Andrew Or] Report REST server response back to the user
40e6095 [Andrew Or] Pass submit parameters through system properties
cbd670b [Andrew Or] Include unknown fields, if any, in server response
9fee16f [Andrew Or] Include server protocol version on mismatch
09f873a [Andrew Or] Fix style
8188e61 [Andrew Or] Upgrade Jackson from 2.3.0 to 2.4.4
37538e0 [Andrew Or] Merge branch 'master' of github.com:apache/spark into rest
9165ae8 [Andrew Or] Fall back to Akka if endpoint was not REST
252d53c [Andrew Or] Clean up server error handling behavior further
c643f64 [Andrew Or] Fix style
bbbd329 [Andrew Or] Merge branch 'master' of github.com:apache/spark into rest
792e112 [Andrew Or] Use specific HTTP response codes on error
f98660b [Andrew Or] Version the protocol and include it in REST URL
721819f [Andrew Or] Provide more REST-like interface for submit/kill/status
581f7bf [Andrew Or] Merge branch 'master' of github.com:apache/spark into rest
9e0d1af [Andrew Or] Move some classes around to reduce number of files (minor)
42e5de4 [Andrew Or] Merge branch 'master' of github.com:apache/spark into rest
1f1c03f [Andrew Or] Use Jackson's DefaultScalaModule to simplify messages
9229433 [Andrew Or] Reduce duplicate naming in REST field
ade28fd [Andrew Or] Clean up REST response output in Spark submit
b2fef8b [Andrew Or] Abstract the success field to the general response
6c57b4b [Andrew Or] Increase timeout in end-to-end tests
bf696ff [Andrew Or] Add checks for enabling REST when using kill/status
7ee6737 [Andrew Or] Merge branch 'master' of github.com:apache/spark into rest
e2f7f5f [Andrew Or] Provide more safeguard against missing fields
9581df7 [Andrew Or] Clean up uses of exceptions
914fdff [Andrew Or] Merge branch 'master' of github.com:apache/spark into rest
e2104e6 [Andrew Or] stable -> rest
3db7379 [Andrew Or] Fix comments and name fields for better error messages
8d43486 [Andrew Or] Replace SubmitRestProtocolAction with class name
df90e8b [Andrew Or] Use Jackson for JSON de/serialization
d7a1f9f [Andrew Or] Fix local cluster tests
efa5e18 [Andrew Or] Merge branch 'master' of github.com:apache/spark into rest
e42c131 [Andrew Or] Add end-to-end tests for standalone REST protocol
837475b [Andrew Or] Show the REST port on the Master UI
d8d3717 [Andrew Or] Use a daemon thread pool for REST server
6568ca5 [Andrew Or] Merge branch 'master' of github.com:apache/spark into rest
77774ba [Andrew Or] Minor fixes
206cae4 [Andrew Or] Refactor and add tests for the REST protocol
63c05b3 [Andrew Or] Remove MASTER as a field (minor)
9e21b72 [Andrew Or] Action -> SparkSubmitAction (minor)
51c5ca6 [Andrew Or] Distinguish client and server side Spark versions
b44e103 [Andrew Or] Implement status requests + fix validation behavior
120ab9d [Andrew Or] Support kill and request driver status through SparkSubmit
544de1d [Andrew Or] Major clean ups in code and comments
e958cae [Andrew Or] Supported nested values in messages
484bd21 [Andrew Or] Specify an ordering for fields in SubmitDriverRequestMessage
6ff088d [Andrew Or] Rename classes to generalize REST protocol
af9d9cb [Andrew Or] Integrate REST protocol in standalone mode
53e7c0e [Andrew Or] Initial client, server, and all the messages
2015-02-06 15:57:06 -08:00
Kostas Sakellis dcd1e42d6b [SPARK-4874] [CORE] Collect record count metrics
Collects record counts for both Input/Output and Shuffle Metrics. For the input/output metrics, it just appends the counter every time the iterators get accessed.

For shuffle on the write side, we count the metrics post aggregation (after a map side combine) and on the read side we count the metrics pre aggregation. This allows both the bytes read/written metrics and the records read/written to line up.

For backwards compatibility, if we deserialize an older event that doesn't have record metrics, we set the metric to -1.

Author: Kostas Sakellis <kostas@cloudera.com>

Closes #4067 from ksakellis/kostas-spark-4874 and squashes the following commits:

bd919be [Kostas Sakellis] Changed 'Records Read' in shuffleReadMetrics json output to 'Total Records Read'
dad4d57 [Kostas Sakellis] Add a comment and check to BlockObjectWriter so that it cannot be reopend.
6f236a1 [Kostas Sakellis] Renamed _recordsWritten in ShuffleWriteMetrics to be more consistent
70620a0 [Kostas Sakellis] CR Feedback
17faa3a [Kostas Sakellis] Removed AtomicLong in favour of using Long
b6f9923 [Kostas Sakellis] Merge AfterNextInterceptingIterator with InterruptableIterator to save a function call
46c8186 [Kostas Sakellis] Combined Bytes and # records into one column
57551c1 [Kostas Sakellis] Conforms to SPARK-3288
6cdb44e [Kostas Sakellis] Removed the generic InterceptingIterator and repalced it with specific implementation
1aa273c [Kostas Sakellis] CR Feedback
1bb78b1 [Kostas Sakellis] [SPARK-4874] [CORE] Collect record count metrics
2015-02-06 14:31:20 -08:00
Marcelo Vanzin 5687bab8fd [SPARK-5600] [core] Clean up FsHistoryProvider test, fix app sort order.
Clean up some test setup code to remove duplicate instantiation of the
provider. Also make sure unfinished apps are sorted correctly.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #4370 from vanzin/SPARK-5600 and squashes the following commits:

0d048d5 [Marcelo Vanzin] Cleanup test code a bit.
2585119 [Marcelo Vanzin] Review feedback.
8b97544 [Marcelo Vanzin] Merge branch 'master' into SPARK-5600
be979e9 [Marcelo Vanzin] Merge branch 'master' into SPARK-5600
298371c [Marcelo Vanzin] [SPARK-5600] [core] Clean up FsHistoryProvider test, fix app sort order.
2015-02-06 14:23:09 -08:00
zsxwing af2a2a263a [SPARK-4361][Doc] Add more docs for Hadoop Configuration
I'm trying to point out reusing a Configuration in these APIs is dangerous. Any better idea?

Author: zsxwing <zsxwing@gmail.com>

Closes #3225 from zsxwing/SPARK-4361 and squashes the following commits:

fe4e3d5 [zsxwing] Add more docs for Hadoop Configuration
2015-02-06 11:51:09 -08:00
Josh Rosen fb6c0cbac4 [HOTFIX] Fix test build break in ExecutorAllocationManagerSuite.
This was caused because #3486 added a new field to ExecutorInfo and #4369
added new tests that created ExecutorInfos.  These patches were merged in
quick succession and were never tested together, hence the compilation error.
2015-02-06 11:48:52 -08:00
Josh Rosen 0d74bd7fd7 [SPARK-5555] Enable UISeleniumSuite tests
This patch enables UISeleniumSuite, a set of tests for the Spark application web UI.  These tests were previously disabled because they were slow, but I think we now have sufficient test time budget that the benefit of enabling them outweighs the time costs.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #4334 from JoshRosen/enable-uiseleniumsuite and squashes the following commits:

4ab9477 [Josh Rosen] Use BeforeAndAfterAll to cleanup WebDriver
71efc72 [Josh Rosen] Update broken UISeleniumSuite tests; use random port #.
a5ab595 [Josh Rosen] Enable UISeleniumSuite tests.
2015-02-06 11:14:58 -08:00
Kostas Sakellis 32e964c410 SPARK-2450 Adds executor log links to Web UI
Adds links to stderr/stdout in the executor tab of the webUI for:
1) Standalone
2) Yarn client
3) Yarn cluster

This tries to add the log url support in a general way so as to make it easy to add support for all the
cluster managers. This is done by using environment variables to pass to the executor the log urls. The
SPARK_LOG_URL_ prefix is used and so additional logs besides stderr/stdout can also be added.

To propagate this information to the UI we use the onExecutorAdded spark listener event.

Although this commit doesn't add log urls when running on a mesos cluster, it should be possible to add using the same mechanism.

Author: Kostas Sakellis <kostas@cloudera.com>
Author: Josh Rosen <joshrosen@databricks.com>

Closes #3486 from ksakellis/kostas-spark-2450 and squashes the following commits:

d190936 [Josh Rosen] Fix a few minor style / formatting nits. Reset listener after each test Don't null listener out at end of main().
8673fe1 [Kostas Sakellis] CR feedback. Hide the log column if there are no logs available
5bf6952 [Kostas Sakellis] [SPARK-2450] [CORE] Adds exeuctor log links to Web UI
2015-02-06 11:13:00 -08:00
Makoto Fukuhara 4cdb26c174 [SPARK-5618][Spark Core][Minor] Optimise utility code.
Author: Makoto Fukuhara <fukuo33@gmail.com>

Closes #4396 from fukuo33/fix-unnecessary-regex and squashes the following commits:

cd07fd6 [Makoto Fukuhara] fix unnecessary regex.
2015-02-06 11:12:34 -08:00
lianhuiwang 6072fcc14e [SPARK-5593][Core]Replace BlockManagerListener with ExecutorListener in ExecutorAllocationListener
More strictly, in ExecutorAllocationListener, we need to replace onBlockManagerAdded, onBlockManagerRemoved with onExecutorAdded,onExecutorRemoved. because at some time, onExecutorAdded and onExecutorRemoved are more accurate to express these meanings. example at SPARK-5529, BlockManager has been removed,but executor is existed.
 andrewor14 sryza

Author: lianhuiwang <lianhuiwang09@gmail.com>

Closes #4369 from lianhuiwang/SPARK-5593 and squashes the following commits:

333367c [lianhuiwang] Replace BlockManagerListener with ExecutorListener in ExecutorAllocationListener
2015-02-06 11:09:37 -08:00
Stephen Haberman 9792bec596 [SPARK-4877] Allow user first classes to extend classes in the parent.
Previously, the classloader isolation was almost too good, such
that if a child class needed to load/reference a class that was
only available in the parent, it could not do so.

This adds tests for that case, the user-first Fake2 class extends
the only-in-parent Fake3 class.

It also sneaks in a fix where only the first stage seemed to work,
and on subsequent stages, a LinkageError happened because classes
from the user-first classpath were getting defined twice.

Author: Stephen Haberman <stephen@exigencecorp.com>

Closes #3725 from stephenh/4877_user_first_parent_inheritance and squashes the following commits:

dabcd35 [Stephen Haberman] [SPARK-4877] Respect userClassPathFirst for the driver code too.
3d0fa7c [Stephen Haberman] [SPARK-4877] Allow user first classes to extend classes in the parent.
2015-02-06 11:03:56 -08:00
Andrew Or fe3740c4c8 [SPARK-5636] Ramp up faster in dynamic allocation
A recent patch #4051 made the initial number default to 0. With this change, any Spark application using dynamic allocation's default settings will ramp up very slowly. Since we never request more executors than needed to saturate the pending tasks, it is safe to ramp up quickly. The current default of 60 may be too slow.

Author: Andrew Or <andrew@databricks.com>

Closes #4409 from andrewor14/dynamic-allocation-interval and squashes the following commits:

d3cc485 [Andrew Or] Lower request interval
2015-02-06 10:55:13 -08:00
Ryan Williams 37d35ab53b [SPARK-5416] init Executor.threadPool before ExecutorSource
Some ExecutorSource metrics can NPE by attempting to reference the
threadpool otherwise.

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

Closes #4212 from ryan-williams/threadpool and squashes the following commits:

236f2ad [Ryan Williams] init Executor.threadPool before ExecutorSource
2015-02-06 12:22:25 +00:00
lianhuiwang ed3aac7912 [SPARK-5470][Core]use defaultClassLoader to load classes in KryoSerializer
Now KryoSerializer load classes of classesToRegister at the time of its initialization. when we set spark.kryo.classesToRegister=class1, it will throw SparkException("Failed to load class to register with Kryo".
because in KryoSerializer's initialization, classLoader cannot include class of user's jars.
we need to use defaultClassLoader of Serializer in newKryo(), because executor will reset defaultClassLoader of Serializer after Serializer's initialization.
thank zzcclp for reporting it to me.

Author: lianhuiwang <lianhuiwang09@gmail.com>

Closes #4258 from lianhuiwang/SPARK-5470 and squashes the following commits:

73b719f [lianhuiwang] do the splitting and filtering during initialization
64cf306 [lianhuiwang] use defaultClassLoader to load classes of classesToRegister in KryoSerializer
2015-02-06 11:00:35 +00:00
Marcelo Vanzin 856928979f [SPARK-5582] [history] Ignore empty log directories.
Empty log directories are not useful at the moment, but if one ends
up showing in the log root, it breaks the code that checks for log
directories.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #4352 from vanzin/SPARK-5582 and squashes the following commits:

1a6a3d4 [Marcelo Vanzin] [SPARK-5582] Fix exception when looking at empty directories.
2015-02-06 10:07:20 +00:00
Patrick Wendell 793dbaef40 SPARK-5557: Explicitly include servlet API in dependencies.
Because of the way we shade jetty, we lose its dependency orbit
in the assembly jar, which includes the javax servlet API's. This
adds back orbit explicitly, using the version that matches
our jetty version.

Author: Patrick Wendell <patrick@databricks.com>

Closes #4411 from pwendell/servlet-api and squashes the following commits:

445f868 [Patrick Wendell] SPARK-5557: Explicitly include servlet API in dependencies.
2015-02-05 18:14:54 -08:00
Cheng Lian a9ed51178c [SPARK-5182] [SPARK-5528] [SPARK-5509] [SPARK-3575] [SQL] Parquet data source improvements
This PR adds three major improvements to Parquet data source:

1.  Partition discovery

    While reading Parquet files resides in Hive style partition directories, `ParquetRelation2` automatically discovers partitioning information and infers partition column types.

    This is also a partial work for [SPARK-5182] [1], which aims to provide first class partitioning support for the data source API.  Related code in this PR can be easily extracted to the data source API level in future versions.

1.  Schema merging

    When enabled, Parquet data source collects schema information from all Parquet part-files and tries to merge them.  Exceptions are thrown when incompatible schemas are detected.  This feature is controlled by data source option `parquet.mergeSchema`, and is enabled by default.

1.  Metastore Parquet table conversion moved to analysis phase

    This greatly simplifies the conversion logic.  `ParquetConversion` strategy can be removed once the old Parquet implementation is removed in the future.

This version of Parquet data source aims to entirely replace the old Parquet implementation.  However, the old version hasn't been removed yet.  Users can fall back to the old version by turning off SQL configuration `spark.sql.parquet.useDataSourceApi`.

Other JIRA tickets fixed as side effects in this PR:

- [SPARK-5509] [3]: `EqualTo` now uses a proper `Ordering` to compare binary types.

- [SPARK-3575] [4]: Metastore schema is now preserved and passed to `ParquetRelation2` via data source option `parquet.metastoreSchema`.

TODO:

- [ ] More test cases for partition discovery
- [x] Fix write path after data source write support (#4294) is merged

      It turned out to be non-trivial to fall back to old Parquet implementation on the write path when Parquet data source is enabled.  Since we're planning to include data source write support in 1.3.0, I simply ignored two test cases involving Parquet insertion for now.

- [ ] Fix outdated comments and documentations

PS: This PR looks big, but more than a half of the changed lines in this PR are trivial changes to test cases. To test Parquet with and without the new data source, almost all Parquet test cases are moved into wrapper driver functions. This introduces hundreds of lines of changes.

[1]: https://issues.apache.org/jira/browse/SPARK-5182
[2]: https://issues.apache.org/jira/browse/SPARK-5528
[3]: https://issues.apache.org/jira/browse/SPARK-5509
[4]: https://issues.apache.org/jira/browse/SPARK-3575

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

Author: Cheng Lian <lian@databricks.com>

Closes #4308 from liancheng/parquet-partition-discovery and squashes the following commits:

b6946e6 [Cheng Lian] Fixes MiMA issues, addresses comments
8232e17 [Cheng Lian] Write support for Parquet data source
a49bd28 [Cheng Lian] Fixes spelling typo in trait name "CreateableRelationProvider"
808380f [Cheng Lian] Fixes issues introduced while rebasing
50dd8d1 [Cheng Lian] Addresses @rxin's comment, fixes UDT schema merging
adf2aae [Cheng Lian] Fixes compilation error introduced while rebasing
4e0175f [Cheng Lian] Fixes Python Parquet API, we need Py4J array to call varargs method
0d8ec1d [Cheng Lian] Adds more test cases
b35c8c6 [Cheng Lian] Fixes some typos and outdated comments
dd704fd [Cheng Lian] Fixes Python Parquet API
596c312 [Cheng Lian] Uses switch to control whether use Parquet data source or not
7d0f7a2 [Cheng Lian] Fixes Metastore Parquet table conversion
a1896c7 [Cheng Lian] Fixes all existing Parquet test suites except for ParquetMetastoreSuite
5654c9d [Cheng Lian] Draft version of Parquet partition discovery and schema merging
2015-02-05 15:29:56 -08:00
Jacek Lewandowski 081ac69f3d SPARK-5548: Fixed a race condition in AkkaUtilsSuite
`Await.result` and `selection.resolveOne` runs the same timeout simultaneously. When `Await.result` timeout is reached first, then `TimeoutException` is thrown. On the other hand, when `selection.resolveOne` timeout is reached first, `ActorNotFoundException` is thrown. This is an obvious race condition and the easiest way to fix it is to increase the timeout of one method to make sure the code fails on the other method first.

Author: Jacek Lewandowski <lewandowski.jacek@gmail.com>

Closes #4343 from jacek-lewandowski/SPARK-5548-1.3 and squashes the following commits:

b9ba47e [Jacek Lewandowski] SPARK-5548: Fixed a race condition in AkkaUtilsSuite
2015-02-05 12:21:40 -08:00
Sandy Ryza c4b1108c3f SPARK-4687. Add a recursive option to the addFile API
This adds a recursive option to the addFile API to satisfy Hive's needs.  It only allows specifying HDFS dirs that will be copied down on every executor.

There are a couple outstanding questions.
* Should we allow specifying local dirs as well?  The best way to do this would probably be to archive them.  The drawback is that it would require a fair bit of code that I don't know of any current use cases for.
* The addFiles implementation has a caching component that I don't entirely understand.  What events are we caching between?  AFAICT it's users calling addFile on the same file in the same app at different times?  Do we want/need to add something similar for addDirectory.
*  The addFiles implementation will check to see if an added file already exists and has the same contents.  I imagine we want the same behavior, so planning to add this unless people think otherwise.

I plan to add some tests if people are OK with the approach.

Author: Sandy Ryza <sandy@cloudera.com>

Closes #3670 from sryza/sandy-spark-4687 and squashes the following commits:

f9fc77f [Sandy Ryza] Josh's comments
70cd24d [Sandy Ryza] Add another test
13da824 [Sandy Ryza] Revert executor changes
38bf94d [Sandy Ryza] Marcelo's comments
ca83849 [Sandy Ryza] Add addFile test
1941be3 [Sandy Ryza] Fix test and avoid HTTP server in local mode
31f15a9 [Sandy Ryza] Use cache recursively and fix some compile errors
0239c3d [Sandy Ryza] Change addDirectory to addFile with recursive
46fe70a [Sandy Ryza] SPARK-4687. Add a addDirectory API
2015-02-05 10:15:55 -08:00
Josh Rosen 9a7ce70eab [SPARK-5411] Allow SparkListeners to be specified in SparkConf and loaded when creating SparkContext
This patch introduces a new configuration option, `spark.extraListeners`, that allows SparkListeners to be specified in SparkConf and registered before the SparkContext is initialized.  From the configuration documentation:

> A comma-separated list of classes that implement SparkListener; when initializing SparkContext, instances of these classes will be created and registered with Spark's listener bus. If a class has a single-argument constructor that accepts a SparkConf, that constructor will be called; otherwise, a zero-argument constructor will be called. If no valid constructor can be found, the SparkContext creation will fail with an exception.

This motivation for this patch is to allow monitoring code to be easily injected into existing Spark programs without having to modify those programs' code.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #4111 from JoshRosen/SPARK-5190-register-sparklistener-in-sc-constructor and squashes the following commits:

8370839 [Josh Rosen] Two minor fixes after merging with master
6e0122c [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-5190-register-sparklistener-in-sc-constructor
1a5b9a0 [Josh Rosen] Remove SPARK_EXTRA_LISTENERS environment variable.
2daff9b [Josh Rosen] Add a couple of explanatory comments for SPARK_EXTRA_LISTENERS.
b9973da [Josh Rosen] Add test to ensure that conf and env var settings are merged, not overriden.
d6f3113 [Josh Rosen] Use getConstructors() instead of try-catch to find right constructor.
d0d276d [Josh Rosen] Move code into setupAndStartListenerBus() method
b22b379 [Josh Rosen] Instantiate SparkListeners from classes listed in configurations.
9c0d8f1 [Josh Rosen] Revert "[SPARK-5190] Allow SparkListeners to be registered before SparkContext starts."
217ecc0 [Josh Rosen] Revert "Add addSparkListener to JavaSparkContext"
25988f3 [Josh Rosen] Add addSparkListener to JavaSparkContext
163ba19 [Josh Rosen] [SPARK-5190] Allow SparkListeners to be registered before SparkContext starts.
2015-02-04 17:18:03 -08:00
Davies Liu 0a89b15685 [SPARK-4939] move to next locality when no pending tasks
Currently, if there are different locality in a task set, the tasks with NODE_LOCAL only get scheduled after all the PROCESS_LOCAL tasks are scheduled and timeout with spark.locality.wait.process (3 seconds by default). In local mode, the LocalScheduler will never call resourceOffer() again once it failed to get a task with same locality, then all the NODE_LOCAL tasks will be never scheduled.

This bug could be reproduced by run example python/streaming/stateful_network_wordcount.py, it will hang after finished a batch with some data.

This patch will check whether there is task for current locality level, if not, it will change to next locality level without waiting for `spark.locality.wait.process` seconds. It works for all locality levels.

Because the list of pending tasks are updated lazily, the check can be false-positive, it means it will not move to next locality level even there is no valid pending tasks, it will wait for timeout.

Author: Davies Liu <davies@databricks.com>

Closes #3779 from davies/local_streaming and squashes the following commits:

2d25fb3 [Davies Liu] Update TaskSetManager.scala
1550668 [Davies Liu] add comment
1c37aac [Davies Liu] address comments
6b13824 [Davies Liu] address comments
906f456 [Davies Liu] Merge branch 'master' of github.com:apache/spark into local_streaming
414e79e [Davies Liu] fix bug, add logging
ff8eabb [Davies Liu] Merge branch 'master' into local_streaming
28d1b3c [Davies Liu] check tasks
9d0ceab [Davies Liu] Merge branch 'master' of github.com:apache/spark into local_streaming
37a2804 [Davies Liu] fix tests
49bda82 [Davies Liu] address comment
d8fb95a [Davies Liu] move to next locality level if no more tasks
2d6ae73 [Davies Liu] add comments
32d363f [Davies Liu] add regression test
7d8c5a5 [Davies Liu] jump to next locality if no pending tasks for executors
2015-02-04 14:22:07 -08:00
Davies Liu ac0b2b788f [SPARK-5588] [SQL] support select/filter by SQL expression
```
df.selectExpr('a + 1', 'abs(age)')
df.filter('age > 3')
df[ df.age > 3 ]
df[ ['age', 'name'] ]
```

Author: Davies Liu <davies@databricks.com>

Closes #4359 from davies/select_expr and squashes the following commits:

d99856b [Davies Liu] support select/filter by SQL expression
2015-02-04 11:34:46 -08:00
Imran Rashid 5aa0f219d1 [SPARK-5574] use given name prefix in dir
https://issues.apache.org/jira/browse/SPARK-5574

very minor, doesn't effect external behavior at all.
Note that after this change, some of these dirs no longer will have "spark" in the name at all.  I could change those locations that do pass in a name prefix to also include "spark", eg. "blockmgr" -> "spark-blockmgr"

Author: Imran Rashid <irashid@cloudera.com>

Closes #4344 from squito/SPARK-5574 and squashes the following commits:

33a84fe [Imran Rashid] use given name prefix in dir
2015-02-04 01:02:20 -08:00
Burak Yavuz 6aed719e50 [SPARK-5341] Use maven coordinates as dependencies in spark-shell and spark-submit
This PR adds support for using maven coordinates as dependencies to spark-shell.
Coordinates can be provided as a comma-delimited string after the flag `--packages`.
Additional remote repositories (like SonaType) can be supplied as a comma-delimited string after the flag
`--repositories`.

Uses the Ivy library to resolve dependencies. Unfortunately the library has no decent documentation, therefore solving more complex dependency issues can be a problem.

pwendell, mateiz, mengxr

**Note: This is still a WIP. The following need to be handled:**
- [x] add docs for the methods
- [x] take local ivy cache path as an argument
- [x] add tests
- [x] add Windows compatibility
- [x] exclude unused Ivy dependencies

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #4215 from brkyvz/SPARK-5341ivy and squashes the following commits:

9215851 [Burak Yavuz] ready to merge
db2a5cc [Burak Yavuz] changed logging to printStream
9dae87f [Burak Yavuz] file separators changed
71c374d [Burak Yavuz] merge conflicts fixed
c08dc9f [Burak Yavuz] fixed merge conflicts
3ada19a [Burak Yavuz] fixed Jenkins error (hopefully) and added comment on oro
43c2290 [Burak Yavuz] fixed that ONE line
231f72f [Burak Yavuz] addressed code review
2cd6562 [Burak Yavuz] Merge branch 'master' of github.com:apache/spark into SPARK-5341ivy
85ec5a3 [Burak Yavuz] added oro as a dependency explicitly
ea44ca4 [Burak Yavuz] add oro back to dependencies
cef0e24 [Burak Yavuz] IntelliJ is just messing things up
97c4a92 [Burak Yavuz] fix more weird IntelliJ formatting
9cf077d [Burak Yavuz] fix weird IntelliJ formatting
dcf5e13 [Burak Yavuz] fix windows command line flags
3a23f21 [Burak Yavuz] excluded ivy dependencies
53423e0 [Burak Yavuz] tests added
3705907 [Burak Yavuz] remove ivy-repo as a command line argument. Use global ivy cache as default
c04d885 [Burak Yavuz] take path to ivy cache as a conf
2edc9b5 [Burak Yavuz] managed to exclude Spark and it's dependencies
a0870af [Burak Yavuz] add docs. remove unnecesary new lines
6645af4 [Burak Yavuz] [SPARK-5341] added base implementation
882c4c8 [Burak Yavuz] added maven dependency download
2015-02-03 22:39:17 -08:00
Davies Liu 83de71c45b [SPARK-4939] revive offers periodically in LocalBackend
The locality timeout assume that the SchedulerBackend can revive offers periodically, but currently LocalBackend did do that, then some job with mixed locality levels in local mode will hang forever.

This PR let LocalBackend revive offers periodically, just like in cluster mode.

Author: Davies Liu <davies@databricks.com>

Closes #4147 from davies/revive and squashes the following commits:

2acdf9d [Davies Liu] Update LocalBackend.scala
3c8ca7c [Davies Liu] Update LocalBackend.scala
d1b60d2 [Davies Liu] address comments from Kay
33ac9bb [Davies Liu] fix build
d0da0d5 [Davies Liu] Merge branch 'master' of github.com:apache/spark into revive
6cf5972 [Davies Liu] fix thread-safety
ed62a31 [Davies Liu] fix scala style
df9008b [Davies Liu] fix typo
bfc1396 [Davies Liu] revive offers periodically in LocalBackend
2015-02-03 22:30:23 -08:00
freeman 242b4f02df [SPARK-4969][STREAMING][PYTHON] Add binaryRecords to streaming
In Spark 1.2 we added a `binaryRecords` input method for loading flat binary data. This format is useful for numerical array data, e.g. in scientific computing applications. This PR adds support for the same format in Streaming applications, where it is similarly useful, especially for streaming time series or sensor data.

Summary of additions
- adding `binaryRecordsStream` to Spark Streaming
- exposing `binaryRecordsStream` in the new PySpark Streaming
- new unit tests in Scala and Python

This required adding an optional Hadoop configuration param to `fileStream` and `FileInputStream`, but was otherwise straightforward.

tdas davies

Author: freeman <the.freeman.lab@gmail.com>

Closes #3803 from freeman-lab/streaming-binary-records and squashes the following commits:

b676534 [freeman] Clarify note
5ff1b75 [freeman] Add note to java streaming context
eba925c [freeman] Simplify notes
c4237b8 [freeman] Add experimental tag
30eba67 [freeman] Add filter and newFilesOnly alongside conf
c2cfa6d [freeman] Expose new version of fileStream with conf in java
34d20ef [freeman] Add experimental tag
14bca9a [freeman] Add experimental tag
b85bffc [freeman] Formatting
47560f4 [freeman] Space formatting
9a3715a [freeman] Refactor to reflect changes to FileInputSuite
7373f73 [freeman] Add note and defensive assertion for byte length
3ceb684 [freeman] Merge remote-tracking branch 'upstream/master' into streaming-binary-records
317b6d1 [freeman] Make test inline
fcb915c [freeman] Formatting
becb344 [freeman] Formatting
d3e75b2 [freeman] Add tests in python
a4324a3 [freeman] Line length
029d49c [freeman] Formatting
1c739aa [freeman] Simpler default arg handling
94d90d0 [freeman] Spelling
2843e9d [freeman] Add params to docstring
8b70fbc [freeman] Reorganization
28bff9b [freeman] Fix missing arg
9398bcb [freeman] Expose optional hadoop configuration
23dd69f [freeman] Tests for binaryRecordsStream
36cb0fd [freeman] Add binaryRecordsStream to scala
fe4e803 [freeman] Add binaryRecordStream to Java API
ecef0eb [freeman] Add binaryRecordsStream to python
8550c26 [freeman] Expose additional argument combination
2015-02-03 22:24:30 -08:00
zsxwing d37978d8aa [SPARK-4795][Core] Redesign the "primitive type => Writable" implicit APIs to make them be activated automatically
Try to redesign the "primitive type => Writable" implicit APIs to make them be activated automatically and without breaking binary compatibility.

However, this PR will breaking the source compatibility if people use `xxxToXxxWritable` occasionally. See the unit test in `graphx`.

Author: zsxwing <zsxwing@gmail.com>

Closes #3642 from zsxwing/SPARK-4795 and squashes the following commits:

914b2d6 [zsxwing] Add implicit back to the Writables methods
0b9017f [zsxwing] Add some docs
a0e8509 [zsxwing] Merge branch 'master' into SPARK-4795
39343de [zsxwing] Fix the unit test
64853af [zsxwing] Reorganize the rest 'implicit' methods in SparkContext
2015-02-03 20:17:12 -08:00
Reynold Xin f7948f3f57 Minor: Fix TaskContext deprecated annotations.
Made a mistake in https://github.com/apache/spark/pull/4324

Author: Reynold Xin <rxin@databricks.com>

Closes #4333 from rxin/taskcontext-deprecate and squashes the following commits:

61c44ee [Reynold Xin] Minor: Fix TaskContext deprecated annotations.
2015-02-03 10:34:16 -08:00
Reynold Xin bebf4c42be [SPARK-5549] Define TaskContext interface in Scala.
So the interface documentation shows up in ScalaDoc.

Author: Reynold Xin <rxin@databricks.com>

Closes #4324 from rxin/TaskContext-scala and squashes the following commits:

2480a17 [Reynold Xin] comment
573756f [Reynold Xin] style fixes and javadoc fixes.
87dd537 [Reynold Xin] [SPARK-5549] Define TaskContext interface in Scala.
2015-02-03 00:46:04 -08:00
Josh Rosen b8ebebeaaa [SPARK-5414] Add SparkFirehoseListener class for consuming all SparkListener events
There isn't a good way to write a SparkListener that receives all SparkListener events and which will be future-compatible (e.g. it will receive events introduced in newer versions of Spark without having to override new methods to process those events).

To address this, this patch adds `SparkFirehoseListener`, a SparkListener implementation that receives all events and dispatches them to a single `onEvent` method (which can be overridden by users).

Author: Josh Rosen <joshrosen@databricks.com>

Closes #4210 from JoshRosen/firehose-listener and squashes the following commits:

223f579 [Josh Rosen] Expand comment to explain rationale for this being a Java class.
ecdfaed [Josh Rosen] Add SparkFirehoseListener class for consuming all SparkListener events.
2015-02-02 23:35:07 -08:00
Thomas Graves c31c36c4a7 [SPARK-3778] newAPIHadoopRDD doesn't properly pass credentials for secure hdfs
.this was https://github.com/apache/spark/pull/2676

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

This affects if someone is trying to access secure hdfs something like:
val lines = {
val hconf = new Configuration()
hconf.set("mapred.input.dir", "mydir")
hconf.set("textinputformat.record.delimiter","\003432\n")
sc.newAPIHadoopRDD(hconf, classOf[TextInputFormat], classOf[LongWritable], classOf[Text])
}

Author: Thomas Graves <tgraves@apache.org>

Closes #4292 from tgravescs/SPARK-3788 and squashes the following commits:

cf3b453 [Thomas Graves] newAPIHadoopRDD doesn't properly pass credentials for secure hdfs on yarn
2015-02-02 22:45:55 -08:00
zsxwing c306555f49 [SPARK-5219][Core] Add locks to avoid scheduling race conditions
Author: zsxwing <zsxwing@gmail.com>

Closes #4019 from zsxwing/SPARK-5219 and squashes the following commits:

36a8b4e [zsxwing] Add locks to avoid race conditions
2015-02-02 21:42:18 -08:00
Patrick Wendell 7930d2bef0 SPARK-3996: Add jetty servlet and continuations.
These are needed transitively from the other Jetty libraries
we include. It was not picked up by unit tests because we
disable the UI.

Author: Patrick Wendell <patrick@databricks.com>

Closes #4323 from pwendell/jetty and squashes the following commits:

d8669da [Patrick Wendell] SPARK-3996: Add jetty servlet and continuations.
2015-02-02 21:01:36 -08:00