Commit graph

4204 commits

Author SHA1 Message Date
Josh Rosen 939ba1f8f6 [SPARK-4835] Disable validateOutputSpecs for Spark Streaming jobs
This patch disables output spec. validation for jobs launched through Spark Streaming, since this interferes with checkpoint recovery.

Hadoop OutputFormats have a `checkOutputSpecs` method which performs certain checks prior to writing output, such as checking whether the output directory already exists.  SPARK-1100 added checks for FileOutputFormat, SPARK-1677 (#947) added a SparkConf configuration to disable these checks, and SPARK-2309 (#1088) extended these checks to run for all OutputFormats, not just FileOutputFormat.

In Spark Streaming, we might have to re-process a batch during checkpoint recovery, so `save` actions may be called multiple times.  In addition to `DStream`'s own save actions, users might use `transform` or `foreachRDD` and call the `RDD` and `PairRDD` save actions.  When output spec. validation is enabled, the second calls to these actions will fail due to existing output.

This patch automatically disables output spec. validation for jobs submitted by the Spark Streaming scheduler.  This is done by using Scala's `DynamicVariable` to propagate the bypass setting without having to mutate SparkConf or introduce a global variable.

Author: Josh Rosen <joshrosen@databricks.com>

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

36eaf35 [Josh Rosen] Add comment explaining use of transform() in test.
6485cf8 [Josh Rosen] Add test case in Streaming; fix bug for transform()
7b3e06a [Josh Rosen] Remove Streaming-specific setting to undo this change; update conf. guide
bf9094d [Josh Rosen] Revise disableOutputSpecValidation() comment to not refer to Spark Streaming.
e581d17 [Josh Rosen] Deduplicate isOutputSpecValidationEnabled logic.
762e473 [Josh Rosen] [SPARK-4835] Disable validateOutputSpecs for Spark Streaming jobs.
2015-01-04 20:26:18 -08:00
Dale 3fddc9468f [SPARK-4787] Stop SparkContext if a DAGScheduler init error occurs
Author: Dale <tigerquoll@outlook.com>

Closes #3809 from tigerquoll/SPARK-4787 and squashes the following commits:

5661e01 [Dale] [SPARK-4787] Ensure that call to stop() doesn't lose the exception by using a finally block.
2172578 [Dale] [SPARK-4787] Stop context properly if an exception occurs during DAGScheduler initialization.
2015-01-04 13:29:13 -08:00
Brennon York b96008d552 [SPARK-794][Core] Remove sleep() in ClusterScheduler.stop
Removed `sleep()` from the `stop()` method of the `TaskSchedulerImpl` class which, from the JIRA ticket, is believed to be a legacy artifact slowing down testing originally introduced in the `ClusterScheduler` class.

Author: Brennon York <brennon.york@capitalone.com>

Closes #3851 from brennonyork/SPARK-794 and squashes the following commits:

04c3e64 [Brennon York] Removed sleep() from the stop() method
2015-01-04 12:40:39 -08:00
Josh Rosen 012839807c [HOTFIX] Bind web UI to ephemeral port in DriverSuite
The job launched by DriverSuite should bind the web UI to an ephemeral port, since it looks like port contention in this test has caused a large number of Jenkins failures when many builds are started simultaneously.  Our tests already disable the web UI, but this doesn't affect subprocesses launched by our tests.  In this case, I've opted to bind to an ephemeral port instead of disabling the UI because disabling features in this test may mask its ability to catch certain bugs.

See also: e24d3a9

Author: Josh Rosen <joshrosen@databricks.com>

Closes #3873 from JoshRosen/driversuite-webui-port and squashes the following commits:

48cd05c [Josh Rosen] [HOTFIX] Bind web UI to ephemeral port in DriverSuite.
2015-01-01 15:03:54 -08:00
Reynold Xin 7749dd6c36 [SPARK-5038] Add explicit return type for implicit functions.
As we learned in #3580, not explicitly typing implicit functions can lead to compiler bugs and potentially unexpected runtime behavior.

This is a follow up PR for rest of Spark (outside Spark SQL). The original PR for Spark SQL can be found at https://github.com/apache/spark/pull/3859

Author: Reynold Xin <rxin@databricks.com>

Closes #3860 from rxin/implicit and squashes the following commits:

73702f9 [Reynold Xin] [SPARK-5038] Add explicit return type for implicit functions.
2014-12-31 17:07:47 -08:00
Josh Rosen e24d3a9a29 [HOTFIX] Disable Spark UI in SparkSubmitSuite tests
This should fix a major cause of build breaks when running many parallel tests.
2014-12-31 14:13:09 -08:00
Brennon York 8e14c5eb55 [SPARK-4298][Core] - The spark-submit cannot read Main-Class from Manifest.
Resolves a bug where the `Main-Class` from a .jar file wasn't being read in properly. This was caused by the fact that the `primaryResource` object was a URI and needed to be normalized through a call to `.getPath` before it could be passed into the `JarFile` object.

Author: Brennon York <brennon.york@capitalone.com>

Closes #3561 from brennonyork/SPARK-4298 and squashes the following commits:

5e0fce1 [Brennon York] Use string interpolation for error messages, moved comment line from original code to above its necessary code segment
14daa20 [Brennon York] pushed mainClass assignment into match statement, removed spurious spaces, removed { } from case statements, removed return values
c6dad68 [Brennon York] Set case statement to support multiple jar URI's and enabled the 'file' URI to load the main-class
8d20936 [Brennon York] updated to reset the error message back to the default
a043039 [Brennon York] updated to split the uri and jar vals
8da7cbf [Brennon York] fixes SPARK-4298
2014-12-31 11:54:10 -08:00
Josh Rosen 352ed6bbe3 [SPARK-1010] Clean up uses of System.setProperty in unit tests
Several of our tests call System.setProperty (or test code which implicitly sets system properties) and don't always reset/clear the modified properties, which can create ordering dependencies between tests and cause hard-to-diagnose failures.

This patch removes most uses of System.setProperty from our tests, since in most cases we can use SparkConf to set these configurations (there are a few exceptions, including the tests of SparkConf itself).

For the cases where we continue to use System.setProperty, this patch introduces a `ResetSystemProperties` ScalaTest mixin class which snapshots the system properties before individual tests and to automatically restores them on test completion / failure.  See the block comment at the top of the ResetSystemProperties class for more details.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #3739 from JoshRosen/cleanup-system-properties-in-tests and squashes the following commits:

0236d66 [Josh Rosen] Replace setProperty uses in two example programs / tools
3888fe3 [Josh Rosen] Remove setProperty use in LocalJavaStreamingContext
4f4031d [Josh Rosen] Add note on why SparkSubmitSuite needs ResetSystemProperties
4742a5b [Josh Rosen] Clarify ResetSystemProperties trait inheritance ordering.
0eaf0b6 [Josh Rosen] Remove setProperty call in TaskResultGetterSuite.
7a3d224 [Josh Rosen] Fix trait ordering
3fdb554 [Josh Rosen] Remove setProperty call in TaskSchedulerImplSuite
bee20df [Josh Rosen] Remove setProperty calls in SparkContextSchedulerCreationSuite
655587c [Josh Rosen] Remove setProperty calls in JobCancellationSuite
3f2f955 [Josh Rosen] Remove System.setProperty calls in DistributedSuite
cfe9cce [Josh Rosen] Remove use of system properties in SparkContextSuite
8783ab0 [Josh Rosen] Remove TestUtils.setSystemProperty, since it is subsumed by the ResetSystemProperties trait.
633a84a [Josh Rosen] Remove use of system properties in FileServerSuite
25bfce2 [Josh Rosen] Use ResetSystemProperties in UtilsSuite
1d1aa5a [Josh Rosen] Use ResetSystemProperties in SizeEstimatorSuite
dd9492b [Josh Rosen] Use ResetSystemProperties in AkkaUtilsSuite
b0daff2 [Josh Rosen] Use ResetSystemProperties in BlockManagerSuite
e9ded62 [Josh Rosen] Use ResetSystemProperties in TaskSchedulerImplSuite
5b3cb54 [Josh Rosen] Use ResetSystemProperties in SparkListenerSuite
0995c4b [Josh Rosen] Use ResetSystemProperties in SparkContextSchedulerCreationSuite
c83ded8 [Josh Rosen] Use ResetSystemProperties in SparkConfSuite
51aa870 [Josh Rosen] Use withSystemProperty in ShuffleSuite
60a63a1 [Josh Rosen] Use ResetSystemProperties in JobCancellationSuite
14a92e4 [Josh Rosen] Use withSystemProperty in FileServerSuite
628f46c [Josh Rosen] Use ResetSystemProperties in DistributedSuite
9e3e0dd [Josh Rosen] Add ResetSystemProperties test fixture mixin; use it in SparkSubmitSuite.
4dcea38 [Josh Rosen] Move withSystemProperty to TestUtils class.
2014-12-30 18:12:20 -08:00
Josh Rosen efa80a531e [SPARK-4882] Register PythonBroadcast with Kryo so that PySpark works with KryoSerializer
This PR fixes an issue where PySpark broadcast variables caused NullPointerExceptions if KryoSerializer was used.  The fix is to register PythonBroadcast with Kryo so that it's deserialized with a KryoJavaSerializer.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #3831 from JoshRosen/SPARK-4882 and squashes the following commits:

0466c7a [Josh Rosen] Register PythonBroadcast with Kryo.
d5b409f [Josh Rosen] Enable registrationRequired, which would have caught this bug.
069d8a7 [Josh Rosen] Add failing test for SPARK-4882
2014-12-30 09:29:52 -08:00
Zhang, Liye 9077e721cd [SPARK-4920][UI] add version on master and worker page for standalone mode
Author: Zhang, Liye <liye.zhang@intel.com>

Closes #3769 from liyezhang556520/spark-4920_WebVersion and squashes the following commits:

3bb7e0d [Zhang, Liye] add version on master and worker page
2014-12-30 09:19:47 -08:00
Yash Datta 9bc0df6804 SPARK-4968: takeOrdered to skip reduce step in case mappers return no partitions
takeOrdered should skip reduce step in case mapped RDDs have no partitions. This prevents the mentioned exception :

4. run query
SELECT * FROM testTable WHERE market = 'market2' ORDER BY End_Time DESC LIMIT 100;
Error trace
java.lang.UnsupportedOperationException: empty collection
at org.apache.spark.rdd.RDD$$anonfun$reduce$1.apply(RDD.scala:863)
at org.apache.spark.rdd.RDD$$anonfun$reduce$1.apply(RDD.scala:863)
at scala.Option.getOrElse(Option.scala:120)
at org.apache.spark.rdd.RDD.reduce(RDD.scala:863)
at org.apache.spark.rdd.RDD.takeOrdered(RDD.scala:1136)

Author: Yash Datta <Yash.Datta@guavus.com>

Closes #3830 from saucam/fix_takeorder and squashes the following commits:

5974d10 [Yash Datta] SPARK-4968: takeOrdered to skip reduce step in case mappers return no partitions
2014-12-29 13:49:45 -08:00
Kousuke Saruta 8d72341ab7 [Minor] Fix a typo of type parameter in JavaUtils.scala
In JavaUtils.scala, thare is a typo of type parameter. In addition, the type information is removed at the time of compile by erasure.

This issue is really minor so I don't  file in JIRA.

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

Closes #3789 from sarutak/fix-typo-in-javautils and squashes the following commits:

e20193d [Kousuke Saruta] Fixed a typo of type parameter
82bc5d9 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into fix-typo-in-javautils
99f6f63 [Kousuke Saruta] Fixed a typo of type parameter in JavaUtils.scala
2014-12-29 12:05:08 -08:00
YanTangZhai 815de54002 [SPARK-4946] [CORE] Using AkkaUtils.askWithReply in MapOutputTracker.askTracker to reduce the chance of the communicating problem
Using AkkaUtils.askWithReply in MapOutputTracker.askTracker to reduce the chance of the communicating problem

Author: YanTangZhai <hakeemzhai@tencent.com>
Author: yantangzhai <tyz0303@163.com>

Closes #3785 from YanTangZhai/SPARK-4946 and squashes the following commits:

9ca6541 [yantangzhai] [SPARK-4946] [CORE] Using AkkaUtils.askWithReply in MapOutputTracker.askTracker to reduce the chance of the communicating problem
e4c2c0a [YanTangZhai] Merge pull request #15 from apache/master
718afeb [YanTangZhai] Merge pull request #12 from apache/master
6e643f8 [YanTangZhai] Merge pull request #11 from apache/master
e249846 [YanTangZhai] Merge pull request #10 from apache/master
d26d982 [YanTangZhai] Merge pull request #9 from apache/master
76d4027 [YanTangZhai] Merge pull request #8 from apache/master
03b62b0 [YanTangZhai] Merge pull request #7 from apache/master
8a00106 [YanTangZhai] Merge pull request #6 from apache/master
cbcba66 [YanTangZhai] Merge pull request #3 from apache/master
cdef539 [YanTangZhai] Merge pull request #1 from apache/master
2014-12-29 11:30:54 -08:00
GuoQiang Li 080ceb771a [SPARK-4952][Core]Handle ConcurrentModificationExceptions in SparkEnv.environmentDetails
Author: GuoQiang Li <witgo@qq.com>

Closes #3788 from witgo/SPARK-4952 and squashes the following commits:

d903529 [GuoQiang Li] Handle ConcurrentModificationExceptions in SparkEnv.environmentDetails
2014-12-26 23:31:29 -08:00
Zhang, Liye 786808abfd [SPARK-4954][Core] add spark version infomation in log for standalone mode
The master and worker spark version may be not the same with Driver spark version. That is because spark Jar file might be replaced for new application without restarting the spark cluster. So there shall log out the spark-version in both Mater and Worker log.

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

Closes #3790 from liyezhang556520/version4Standalone and squashes the following commits:

e05e1e3 [Zhang, Liye] add spark version infomation in log for standalone mode
2014-12-26 23:24:22 -08:00
Sean Owen 29fabb1b52 SPARK-4297 [BUILD] Build warning fixes omnibus
There are a number of warnings generated in a normal, successful build right now. They're mostly Java unchecked cast warnings, which can be suppressed. But there's a grab bag of other Scala language warnings and so on that can all be easily fixed. The forthcoming PR fixes about 90% of the build warnings I see now.

Author: Sean Owen <sowen@cloudera.com>

Closes #3157 from srowen/SPARK-4297 and squashes the following commits:

8c9e469 [Sean Owen] Suppress unchecked cast warnings, and several other build warning fixes
2014-12-24 13:32:51 -08:00
Kousuke Saruta 199e59aacd [SPARK-4881][Minor] Use SparkConf#getBoolean instead of get().toBoolean
It's really a minor issue.

In ApplicationMaster, there is code like as follows.

    val preserveFiles = sparkConf.get("spark.yarn.preserve.staging.files", "false").toBoolean

I think, the code can be simplified like as follows.

    val preserveFiles = sparkConf.getBoolean("spark.yarn.preserve.staging.files", false)

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

Closes #3733 from sarutak/SPARK-4881 and squashes the following commits:

1771430 [Kousuke Saruta] Modified the code like sparkConf.get(...).toBoolean to sparkConf.getBoolean(...)
c63daa0 [Kousuke Saruta] Simplified code
2014-12-23 19:14:34 -08:00
Marcelo Vanzin 7e2deb71c4 [SPARK-4606] Send EOF to child JVM when there's no more data to read.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #3460 from vanzin/SPARK-4606 and squashes the following commits:

031207d [Marcelo Vanzin] [SPARK-4606] Send EOF to child JVM when there's no more data to read.
2014-12-23 16:07:59 -08:00
Liang-Chi Hsieh 96281cd0c3 [SPARK-4913] Fix incorrect event log path
SPARK-2261 uses a single file to log events for an app. `eventLogDir` in `ApplicationDescription` is replaced with `eventLogFile`. However, `ApplicationDescription` in `SparkDeploySchedulerBackend` is initialized with `SparkContext`'s `eventLogDir`. It is just the log directory, not the actual log file path. `Master.rebuildSparkUI` can not correctly rebuild a new SparkUI for the app.

Because the `ApplicationDescription` is remotely registered with `Master` and the app's id is then generated in `Master`, we can not get the app id in advance before registration. So the received description needs to be modified with correct `eventLogFile` value.

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

Closes #3755 from viirya/fix_app_logdir and squashes the following commits:

5e0ea35 [Liang-Chi Hsieh] Revision for comment.
b5730a1 [Liang-Chi Hsieh] Fix incorrect event log path.

Closes #3777 (a duplicate PR for the same JIRA)
2014-12-23 14:58:44 -08:00
Andrew Or 27c5399f4d [SPARK-4730][YARN] Warn against deprecated YARN settings
See https://issues.apache.org/jira/browse/SPARK-4730.

Author: Andrew Or <andrew@databricks.com>

Closes #3590 from andrewor14/yarn-settings and squashes the following commits:

36e0753 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-settings
dcd1316 [Andrew Or] Warn against deprecated YARN settings
2014-12-23 14:28:36 -08:00
Marcelo Vanzin dd155369a0 [SPARK-4834] [standalone] Clean up application files after app finishes.
Commit 7aacb7bfa added support for sharing downloaded files among multiple
executors of the same app. That works great in Yarn, since the app's directory
is cleaned up after the app is done.

But Spark standalone mode didn't do that, so the lock/cache files created
by that change were left around and could eventually fill up the disk hosting
/tmp.

To solve that, create app-specific directories under the local dirs when
launching executors. Multiple executors launched by the same Worker will
use the same app directories, so they should be able to share the downloaded
files. When the application finishes, a new message is sent to all workers
telling them the application has finished; once that message has been received,
and all executors registered for the application shut down, then those
directories will be cleaned up by the Worker.

Note: Unit testing this is hard (if even possible), since local-cluster mode
doesn't seem to leave the Master/Worker daemons running long enough after
`sc.stop()` is called for the clean up protocol to take effect.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #3705 from vanzin/SPARK-4834 and squashes the following commits:

b430534 [Marcelo Vanzin] Remove seemingly unnecessary synchronization.
50eb4b9 [Marcelo Vanzin] Review feedback.
c0e5ea5 [Marcelo Vanzin] [SPARK-4834] [standalone] Clean up application files after app finishes.
2014-12-23 12:02:08 -08:00
zsxwing c233ab3d8d [SPARK-4818][Core] Add 'iterator' to reduce memory consumed by join
In Scala, `map` and `flatMap` of `Iterable` will copy the contents of `Iterable` to a new `Seq`. Such as,
```Scala
  val iterable = Seq(1, 2, 3).map(v => {
    println(v)
    v
  })
  println("Iterable map done")

  val iterator = Seq(1, 2, 3).iterator.map(v => {
    println(v)
    v
  })
  println("Iterator map done")
```
outputed
```
1
2
3
Iterable map done
Iterator map done
```
So we should use 'iterator' to reduce memory consumed by join.

Found by Johannes Simon in http://mail-archives.apache.org/mod_mbox/spark-user/201412.mbox/%3C5BE70814-9D03-4F61-AE2C-0D63F2DE4446%40mail.de%3E

Author: zsxwing <zsxwing@gmail.com>

Closes #3671 from zsxwing/SPARK-4824 and squashes the following commits:

48ee7b9 [zsxwing] Remove the explicit types
95d59d6 [zsxwing] Add 'iterator' to reduce memory consumed by join
2014-12-22 14:26:28 -08:00
genmao.ygm de9d7d2b5b [SPARK-4920][UI]:current spark version in UI is not striking.
It is not convenient to see the Spark version. We can keep the same style with Spark website.

![spark_version](https://cloud.githubusercontent.com/assets/7402327/5527025/1c8c721c-8a35-11e4-8d6a-2734f3c6bdf8.jpg)

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

Closes #3763 from uncleGen/master-clean-141222 and squashes the following commits:

0dcb9a9 [genmao.ygm] [SPARK-4920][UI]:current spark version in UI is not striking.
2014-12-22 14:14:39 -08:00
Kostas Sakellis 7c0ed13d29 [SPARK-4079] [CORE] Consolidates Errors if a CompressionCodec is not available
This commit consolidates some of the exceptions thrown if compression codecs are not available. If a bad configuration string was passed in, a ClassNotFoundException was through. Also, if Snappy was not available, it would throw an InvocationTargetException when the codec was being used (not when it was being initialized). Now, an IllegalArgumentException is thrown when a codec is not available at creation time - either because the class does not exist or the codec itself is not available in the system. This will allow us to have a better message and fail faster.

Author: Kostas Sakellis <kostas@cloudera.com>

Closes #3119 from ksakellis/kostas-spark-4079 and squashes the following commits:

9709c7c [Kostas Sakellis] Removed unnecessary Logging class
63bfdd0 [Kostas Sakellis] Removed isAvailable to preserve binary compatibility
1d0ef2f [Kostas Sakellis] [SPARK-4079] [CORE] Added more information to exception
64f3d27 [Kostas Sakellis] [SPARK-4079] [CORE] Code review feedback
52dfa8f [Kostas Sakellis] [SPARK-4079] [CORE] Default to LZF if Snappy not available
2014-12-22 13:07:01 -08:00
Takeshi Yamamuro fb8e85e80e [SPARK-4733] Add missing prameter comments in ShuffleDependency
Add missing Javadoc comments in ShuffleDependency.

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

Closes #3594 from maropu/DependencyJavadocFix and squashes the following commits:

32129b4 [Takeshi Yamamuro] Fix comments in @aggregator and @mapSideCombine
303c75d [Takeshi Yamamuro] [SPARK-4733] Add missing prameter comments in ShuffleDependency
2014-12-22 12:19:23 -08:00
Zhang, Liye 39272c8cdb [SPARK-4870] Add spark version to driver log
Author: Zhang, Liye <liye.zhang@intel.com>

Closes #3717 from liyezhang556520/version2Log and squashes the following commits:

ccd30d7 [Zhang, Liye] delete log in sparkConf
330f70c [Zhang, Liye] move the log from SaprkConf to SparkContext
96dc115 [Zhang, Liye] remove curly brace
e833330 [Zhang, Liye] add spark version to driver log
2014-12-22 11:38:28 -08:00
zsxwing 93b2f3a882 [SPARK-4918][Core] Reuse Text in saveAsTextFile
Reuse Text in saveAsTextFile to reduce GC.

/cc rxin

Author: zsxwing <zsxwing@gmail.com>

Closes #3762 from zsxwing/SPARK-4918 and squashes the following commits:

59f03eb [zsxwing] Reuse Text in saveAsTextFile
2014-12-22 11:20:00 -08:00
zsxwing 6ee6aa70b7 [SPARK-2075][Core] Make the compiler generate same bytes code for Hadoop 1.+ and Hadoop 2.+
`NullWritable` is a `Comparable` rather than `Comparable[NullWritable]` in Hadoop 1.+, so the compiler cannot find an implicit Ordering for it. It will generate different anonymous classes for `saveAsTextFile` in Hadoop 1.+ and Hadoop 2.+. Therefore, here we provide an Ordering for NullWritable so that the compiler will generate same codes.

I used the following commands to confirm the generated byte codes are some.
```
mvn -Dhadoop.version=1.2.1 -DskipTests clean package -pl core -am
javap -private -c -classpath core/target/scala-2.10/classes org.apache.spark.rdd.RDD > ~/hadoop1.txt

mvn -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -DskipTests clean package -pl core -am
javap -private -c -classpath core/target/scala-2.10/classes org.apache.spark.rdd.RDD > ~/hadoop2.txt

diff ~/hadoop1.txt ~/hadoop2.txt
```

However, the compiler will generate different codes for the classes which call methods of `JobContext/TaskAttemptContext`. `JobContext/TaskAttemptContext` is a class in Hadoop 1.+, and calling its method will use `invokevirtual`, while it's an interface in Hadoop 2.+, and will use `invokeinterface`.

To fix it, we can use reflection to call `JobContext/TaskAttemptContext.getConfiguration`.

Author: zsxwing <zsxwing@gmail.com>

Closes #3740 from zsxwing/SPARK-2075 and squashes the following commits:

39d9df2 [zsxwing] Fix the code style
e4ad8b5 [zsxwing] Use null for the implicit Ordering
734bac9 [zsxwing] Explicitly set the implicit parameters
ca03559 [zsxwing] Use reflection to access JobContext/TaskAttemptContext.getConfiguration
fa40db0 [zsxwing] Add an Ordering for NullWritable to make the compiler generate same byte codes for RDD
2014-12-21 22:10:19 -08:00
Sean Owen c6a3c0d505 SPARK-4910 [CORE] build failed (use of FileStatus.isFile in Hadoop 1.x)
Fix small Hadoop 1 compile error from SPARK-2261. In Hadoop 1.x, all we have is FileStatus.isDir, so these "is file" assertions are changed to "is not a dir". This is how similar checks are done so far in the code base.

Author: Sean Owen <sowen@cloudera.com>

Closes #3754 from srowen/SPARK-4910 and squashes the following commits:

52c5e4e [Sean Owen] Fix small Hadoop 1 compile error from SPARK-2261
2014-12-21 13:16:57 -08:00
huangzhaowei a764960b3b [Minor] Build Failed: value defaultProperties not found
Mvn Build Failed: value defaultProperties not found .Maybe related to this pr:
1d648123a7
andrewor14 can you look at this problem?

Author: huangzhaowei <carlmartinmax@gmail.com>

Closes #3749 from SaintBacchus/Mvn-Build-Fail and squashes the following commits:

8e2917c [huangzhaowei] Build Failed: value defaultProperties not found
2014-12-19 23:32:56 -08:00
Kanwaljit Singh 1d648123a7 SPARK-2641: Passing num executors to spark arguments from properties file
Since we can set spark executor memory and executor cores using property file, we must also be allowed to set the executor instances.

Author: Kanwaljit Singh <kanwaljit.singh@guavus.com>

Closes #1657 from kjsingh/branch-1.0 and squashes the following commits:

d8a5a12 [Kanwaljit Singh] SPARK-2641: Fixing how spark arguments are loaded from properties file for num executors

Conflicts:
	core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
2014-12-19 19:27:23 -08:00
Marcelo Vanzin 456451911d [SPARK-2261] Make event logger use a single file.
Currently the event logger uses a directory and several files to
describe an app's event log, all but one of which are empty. This
is not very HDFS-friendly, since creating lots of nodes in HDFS
(especially when they don't contain any data) is frowned upon due
to the node metadata being kept in the NameNode's memory.

Instead, add a header section to the event log file that contains metadata
needed to read the events. This metadata includes things like the Spark
version (for future code that may need it for backwards compatibility) and
the compression codec used for the event data.

With the new approach, aside from reducing the load on the NN, there's
also a lot less remote calls needed when reading the log directory.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #1222 from vanzin/hist-server-single-log and squashes the following commits:

cc8f5de [Marcelo Vanzin] Store header in plain text.
c7e6123 [Marcelo Vanzin] Update comment.
59c561c [Marcelo Vanzin] Review feedback.
216c5a3 [Marcelo Vanzin] Review comments.
dce28e9 [Marcelo Vanzin] Fix log overwrite test.
f91c13e [Marcelo Vanzin] Handle "spark.eventLog.overwrite", and add unit test.
346f0b4 [Marcelo Vanzin] Review feedback.
ed0023e [Marcelo Vanzin] Merge branch 'master' into hist-server-single-log
3f4500f [Marcelo Vanzin] Unit test for SPARK-3697.
45c7a1f [Marcelo Vanzin] Version of SPARK-3697 for this branch.
b3ee30b [Marcelo Vanzin] Merge branch 'master' into hist-server-single-log
a6d5c50 [Marcelo Vanzin] Merge branch 'master' into hist-server-single-log
16fd491 [Marcelo Vanzin] Use unique log directory for each codec.
0ef3f70 [Marcelo Vanzin] Merge branch 'master' into hist-server-single-log
d93c44a [Marcelo Vanzin] Add a newline to make the header more readable.
9e928ba [Marcelo Vanzin] Add types.
bd6ba8c [Marcelo Vanzin] Review feedback.
a624a89 [Marcelo Vanzin] Merge branch 'master' into hist-server-single-log
04364dc [Marcelo Vanzin] Merge branch 'master' into hist-server-single-log
bb7c2d3 [Marcelo Vanzin] Fix scalastyle warning.
16661a3 [Marcelo Vanzin] Simplify some internal code.
cc6bce4 [Marcelo Vanzin] Some review feedback.
a722184 [Marcelo Vanzin] Do not encode metadata in log file name.
3700586 [Marcelo Vanzin] Restore log flushing.
f677930 [Marcelo Vanzin] Fix botched rebase.
ae571fa [Marcelo Vanzin] Fix end-to-end event logger test.
9db0efd [Marcelo Vanzin] Show prettier name in UI.
8f42274 [Marcelo Vanzin] Make history server parse old-style log directories.
6251dd7 [Marcelo Vanzin] Make event logger use a single file.
2014-12-19 18:23:42 -08:00
Ryan Williams 7981f96976 [SPARK-4896] don’t redundantly overwrite executor JAR deps
Author: Ryan Williams <ryan.blake.williams@gmail.com>

Closes #2848 from ryan-williams/fetch-file and squashes the following commits:

c14daff [Ryan Williams] Fix copy that was changed to a move inadvertently
8e39c16 [Ryan Williams] code review feedback
788ed41 [Ryan Williams] don’t redundantly overwrite executor JAR deps
2014-12-19 15:24:41 -08:00
Ryan Williams cdb2c645ab [SPARK-4889] update history server example cmds
Author: Ryan Williams <ryan.blake.williams@gmail.com>

Closes #3736 from ryan-williams/hist and squashes the following commits:

421d8ff [Ryan Williams] add another random typo fix
76d6a4c [Ryan Williams] remove hdfs example
a2d0f82 [Ryan Williams] code review feedback
9ca7629 [Ryan Williams] [SPARK-4889] update history server example cmds
2014-12-19 13:56:04 -08:00
Reynold Xin 336cd341ee Small refactoring to pass SparkEnv into Executor rather than creating SparkEnv in Executor.
This consolidates some code path and makes constructor arguments simpler for a few classes.

Author: Reynold Xin <rxin@databricks.com>

Closes #3738 from rxin/sparkEnvDepRefactor and squashes the following commits:

82e02cc [Reynold Xin] Fixed couple bugs.
217062a [Reynold Xin] Code review feedback.
bd00af7 [Reynold Xin] Small refactoring to pass SparkEnv into Executor rather than creating SparkEnv in Executor.
2014-12-19 12:51:12 -08:00
Sandy Ryza 283263ffaa SPARK-3428. TaskMetrics for running tasks is missing GC time metrics
Author: Sandy Ryza <sandy@cloudera.com>

Closes #3684 from sryza/sandy-spark-3428 and squashes the following commits:

cb827fe [Sandy Ryza] SPARK-3428. TaskMetrics for running tasks is missing GC time metrics
2014-12-18 22:40:44 -08:00
Liang-Chi Hsieh d7fc69a8b5 [SPARK-4674] Refactor getCallSite
The current version of `getCallSite` visits the collection of `StackTraceElement` twice. However, it is unnecessary since we can perform our work with a single visit. We also do not need to keep filtered `StackTraceElement`.

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

Closes #3532 from viirya/refactor_getCallSite and squashes the following commits:

62aa124 [Liang-Chi Hsieh] Fix style.
e741017 [Liang-Chi Hsieh] Refactor getCallSite.
2014-12-18 21:41:02 -08:00
Andrew Or 9804a759b6 [SPARK-4754] Refactor SparkContext into ExecutorAllocationClient
This is such that the `ExecutorAllocationManager` does not take in the `SparkContext` with all of its dependencies as an argument. This prevents future developers of this class to tie down this class further with the `SparkContext`, which has really become quite a monstrous object.

cc'ing pwendell who originally suggested this, and JoshRosen who may have thoughts about the trait mix-in style of `SparkContext`.

Author: Andrew Or <andrew@databricks.com>

Closes #3614 from andrewor14/dynamic-allocation-sc and squashes the following commits:

187070d [Andrew Or] Merge branch 'master' of github.com:apache/spark into dynamic-allocation-sc
59baf6c [Andrew Or] Merge branch 'master' of github.com:apache/spark into dynamic-allocation-sc
347a348 [Andrew Or] Refactor SparkContext into ExecutorAllocationClient
2014-12-18 17:38:33 -08:00
Aaron Davidson 105293a7d0 [SPARK-4837] NettyBlockTransferService should use spark.blockManager.port config
This is used in NioBlockTransferService here:
https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala#L66

Author: Aaron Davidson <aaron@databricks.com>

Closes #3688 from aarondav/SPARK-4837 and squashes the following commits:

ebd2007 [Aaron Davidson] [SPARK-4837] NettyBlockTransferService should use spark.blockManager.port config
2014-12-18 16:43:16 -08:00
Ivan Vergiliev f9f58b9a01 SPARK-4743 - Use SparkEnv.serializer instead of closureSerializer in aggregateByKey and foldByKey
Author: Ivan Vergiliev <ivan@leanplum.com>

Closes #3605 from IvanVergiliev/change-serializer and squashes the following commits:

a49b7cf [Ivan Vergiliev] Use serializer instead of closureSerializer in aggregate/foldByKey.
2014-12-18 16:29:36 -08:00
Madhu Siddalingaiah d5a596d418 [SPARK-4884]: Improve Partition docs
Rewording was based on this discussion: http://apache-spark-developers-list.1001551.n3.nabble.com/RDD-data-flow-td9804.html
This is the associated JIRA ticket: https://issues.apache.org/jira/browse/SPARK-4884

Author: Madhu Siddalingaiah <madhu@madhu.com>

Closes #3722 from msiddalingaiah/master and squashes the following commits:

79e679f [Madhu Siddalingaiah] [DOC]: improve documentation
51d14b9 [Madhu Siddalingaiah] Merge remote-tracking branch 'upstream/master'
38faca4 [Madhu Siddalingaiah] Merge remote-tracking branch 'upstream/master'
cbccbfe [Madhu Siddalingaiah] Documentation: replace <b> with <code> (again)
332f7a2 [Madhu Siddalingaiah] Documentation: replace <b> with <code>
cd2b05a [Madhu Siddalingaiah] Merge remote-tracking branch 'upstream/master'
0fc12d7 [Madhu Siddalingaiah] Documentation: add description for repartitionAndSortWithinPartitions
2014-12-18 16:00:53 -08:00
Ilya Ganelin 3720057b8e [SPARK-3607] ConnectionManager threads.max configs on the thread pools don't work
Hi all - cleaned up the code to get rid of the unused parameter and added some discussion of the ThreadPoolExecutor parameters to explain why we can use a single threadCount instead of providing a min/max.

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

Closes #3664 from ilganeli/SPARK-3607C and squashes the following commits:

3c05690 [Ilya Ganelin] Updated documentation and refactored code to extract shared variables
2014-12-18 12:53:18 -08:00
Saisai Shao cf50631a66 [SPARK-4595][Core] Fix MetricsServlet not work issue
`MetricsServlet` handler should be added to the web UI after initialized by `MetricsSystem`, otherwise servlet handler cannot be attached.

Author: Saisai Shao <saisai.shao@intel.com>
Author: Josh Rosen <joshrosen@databricks.com>
Author: jerryshao <saisai.shao@intel.com>

Closes #3444 from jerryshao/SPARK-4595 and squashes the following commits:

434d17e [Saisai Shao] Merge pull request #10 from JoshRosen/metrics-system-cleanup
87a2292 [Josh Rosen] Guard against misuse of MetricsSystem methods.
f779fe0 [jerryshao] Fix MetricsServlet not work issue
2014-12-17 11:47:44 -08:00
Davies Liu ed362008f0 [SPARK-4437] update doc for WholeCombineFileRecordReader
update doc for WholeCombineFileRecordReader

Author: Davies Liu <davies@databricks.com>
Author: Josh Rosen <joshrosen@databricks.com>

Closes #3301 from davies/fix_doc and squashes the following commits:

1d7422f [Davies Liu] Merge pull request #2 from JoshRosen/whole-text-file-cleanup
dc3d21a [Josh Rosen] More genericization in ConfigurableCombineFileRecordReader.
95d13eb [Davies Liu] address comment
bf800b9 [Davies Liu] update doc for WholeCombineFileRecordReader
2014-12-16 11:19:36 -08:00
meiyoula c7628771da [SPARK-4792] Add error message when making local dir unsuccessfully
Author: meiyoula <1039320815@qq.com>

Closes #3635 from XuTingjun/master and squashes the following commits:

dd1c66d [meiyoula] when old is deleted, it will throw an exception where call it
2a55bc2 [meiyoula] Update DiskBlockManager.scala
1483a4a [meiyoula] Delete multiple retries to make dir
67f7902 [meiyoula] Try some times to make dir maybe more reasonable
1c51a0c [meiyoula] Update DiskBlockManager.scala
2014-12-15 22:30:18 -08:00
wangfei 5c24759ddc [Minor][Core] fix comments in MapOutputTracker
Using driver and executor in the comments of ```MapOutputTracker``` is more clear.

Author: wangfei <wangfei1@huawei.com>

Closes #3700 from scwf/commentFix and squashes the following commits:

aa68524 [wangfei] master and worker should be driver and executor
2014-12-15 16:46:43 -08:00
Sean Owen 2a28bc6100 SPARK-785 [CORE] ClosureCleaner not invoked on most PairRDDFunctions
This looked like perhaps a simple and important one. `combineByKey` looks like it should clean its arguments' closures, and that in turn covers apparently all remaining functions in `PairRDDFunctions` which delegate to it.

Author: Sean Owen <sowen@cloudera.com>

Closes #3690 from srowen/SPARK-785 and squashes the following commits:

8df68fe [Sean Owen] Clean context of most remaining functions in PairRDDFunctions, which ultimately call combineByKey
2014-12-15 16:06:15 -08:00
Ryan Williams 8176b7a02e [SPARK-4668] Fix some documentation typos.
Author: Ryan Williams <ryan.blake.williams@gmail.com>

Closes #3523 from ryan-williams/tweaks and squashes the following commits:

d2eddaa [Ryan Williams] code review feedback
ce27fc1 [Ryan Williams] CoGroupedRDD comment nit
c6cfad9 [Ryan Williams] remove unnecessary if statement
b74ea35 [Ryan Williams] comment fix
b0221f0 [Ryan Williams] fix a gendered pronoun
c71ffed [Ryan Williams] use names on a few boolean parameters
89954aa [Ryan Williams] clarify some comments in {Security,Shuffle}Manager
e465dac [Ryan Williams] Saved building-spark.md with Dillinger.io
83e8358 [Ryan Williams] fix pom.xml typo
dc4662b [Ryan Williams] typo fixes in tuning.md, configuration.md
2014-12-15 14:52:17 -08:00
Ilya Ganelin 38703bbca8 [SPARK-1037] The name of findTaskFromList & findTask in TaskSetManager.scala is confusing
Hi all - I've renamed the methods referenced in this JIRA to clarify that they modify the provided arrays (find vs. deque).

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

Closes #3665 from ilganeli/SPARK-1037B and squashes the following commits:

64c177c [Ilya Ganelin] Renamed deque to dequeue
f27d85e [Ilya Ganelin] Renamed private methods to clarify that they modify the provided parameters
683482a [Ilya Ganelin] Renamed private methods to clarify that they modify the provided parameters
2014-12-15 14:51:15 -08:00
Zhang, Liye 57d37f9c71 [CORE]codeStyle: uniform ConcurrentHashMap define in StorageLevel.scala with other places
Author: Zhang, Liye <liye.zhang@intel.com>

Closes #2793 from liyezhang556520/uniformHashMap and squashes the following commits:

5884735 [Zhang, Liye] [CORE]codeStyle: uniform ConcurrentHashMap define in StorageLevel.scala
2014-12-10 20:44:59 -08:00
Andrew Or 4f93d0cabe [SPARK-4759] Fix driver hanging from coalescing partitions
The driver hangs sometimes when we coalesce RDD partitions. See JIRA for more details and reproduction.

This is because our use of empty string as default preferred location in `CoalescedRDDPartition` causes the `TaskSetManager` to schedule the corresponding task on host `""` (empty string). The intended semantics here, however, is that the partition does not have a preferred location, and the TSM should schedule the corresponding task accordingly.

Author: Andrew Or <andrew@databricks.com>

Closes #3633 from andrewor14/coalesce-preferred-loc and squashes the following commits:

e520d6b [Andrew Or] Oops
3ebf8bd [Andrew Or] A few comments
f370a4e [Andrew Or] Fix tests
2f7dfb6 [Andrew Or] Avoid using empty string as default preferred location
2014-12-10 14:27:53 -08:00
Ilya Ganelin 447ae2de5d [SPARK-4569] Rename 'externalSorting' in Aggregator
Hi all - I've renamed the unhelpfully named variable and added a comment clarifying what's actually happening.

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

Closes #3666 from ilganeli/SPARK-4569B and squashes the following commits:

1810394 [Ilya Ganelin] [SPARK-4569] Rename 'externalSorting' in Aggregator
e2d2092 [Ilya Ganelin] [SPARK-4569] Rename 'externalSorting' in Aggregator
d7cefec [Ilya Ganelin] [SPARK-4569] Rename 'externalSorting' in Aggregator
5b3f39c [Ilya Ganelin] [SPARK-4569] Rename  in Aggregator
2014-12-10 14:19:37 -08:00
Andrew Or faa8fd8178 [SPARK-4215] Allow requesting / killing executors only in YARN mode
Currently this doesn't do anything in other modes, so we might as well just disable it rather than having the user mistakenly rely on it.

Author: Andrew Or <andrew@databricks.com>

Closes #3615 from andrewor14/dynamic-allocation-yarn-only and squashes the following commits:

ce6487a [Andrew Or] Allow requesting / killing executors only in YARN mode
2014-12-10 12:48:24 -08:00
Kousuke Saruta 0fc637b4c2 [SPARK-4329][WebUI] HistoryPage pagenation
Current HistoryPage have links only to previous page or next page.
I suggest to add index to access history pages easily.

I implemented like following pics.

If there are many pages, current page +/- N pages, head page and last page are indexed.

![2014-11-10 16 13 25](https://cloud.githubusercontent.com/assets/4736016/4986246/9c7bbac4-6937-11e4-8695-8634d039d5b6.png)
![2014-11-10 16 03 21](https://cloud.githubusercontent.com/assets/4736016/4986210/3951bb74-6937-11e4-8b4e-9f90d266d736.png)
![2014-11-10 16 03 39](https://cloud.githubusercontent.com/assets/4736016/4986211/3b196ad8-6937-11e4-9f81-74bc0a6dad5b.png)
![2014-11-10 16 03 49](https://cloud.githubusercontent.com/assets/4736016/4986213/40686138-6937-11e4-86c0-41100f0404f6.png)
![2014-11-10 16 04 04](https://cloud.githubusercontent.com/assets/4736016/4986215/4326c9b4-6937-11e4-87ac-0f30c86ec6e3.png)

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

Closes #3194 from sarutak/history-page-indexing and squashes the following commits:

15d3d2d [Kousuke Saruta] Simplified code
c93932e [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into history-page-indexing
1c2f605 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into history-page-indexing
76b05e3 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into history-page-indexing
b2240f8 [Kousuke Saruta] Fixed style
ec7922e [Kousuke Saruta] Simplified code
755a004 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into history-page-indexing
cfa242b [Kousuke Saruta] Added index to HistoryPage
2014-12-10 12:30:45 -08:00
Nathan Kronenfeld 94b377f944 [SPARK-4772] Clear local copies of accumulators as soon as we're done with them
Accumulators keep thread-local copies of themselves.  These copies were only cleared at the beginning of a task.  This meant that (a) the memory they used was tied up until the next task ran on that thread, and (b) if a thread died, the memory it had used for accumulators was locked up forever on that worker.

This PR clears the thread-local copies of accumulators at the end of each task, in the tasks finally block, to make sure they are cleaned up between tasks.  It also stores them in a ThreadLocal object, so that if, for some reason, the thread dies, any memory they are using at the time should be freed up.

Author: Nathan Kronenfeld <nkronenfeld@oculusinfo.com>

Closes #3570 from nkronenfeld/Accumulator-Improvements and squashes the following commits:

a581f3f [Nathan Kronenfeld] Change Accumulators to private[spark] instead of adding mima exclude to get around false positive in mima tests
b6c2180 [Nathan Kronenfeld] Include MiMa exclude as per build error instructions - this version incompatibility should be irrelevent, as it will only surface if a master is talking to a worker running a different version of spark.
537baad [Nathan Kronenfeld] Fuller refactoring as intended, incorporating JR's suggestions for ThreadLocal localAccums, and keeping clear(), but also calling it in tasks' finally block, rather than just at the beginning of the task.
39a82f2 [Nathan Kronenfeld] Clear local copies of accumulators as soon as we're done with them
2014-12-09 23:53:17 -08:00
Josh Rosen f79c1cfc99 [Minor] Use <sup> tag for help icon in web UI page header
This small commit makes the `(?)` web UI help link into a superscript, which should address feedback that the current design makes it look like an error occurred or like information is missing.

Before:

![image](https://cloud.githubusercontent.com/assets/50748/5370611/a3ed0034-7fd9-11e4-870f-05bd9faad5b9.png)

After:

![image](https://cloud.githubusercontent.com/assets/50748/5370602/6c5ca8d6-7fd9-11e4-8d1a-568d71290aa7.png)

Author: Josh Rosen <joshrosen@databricks.com>

Closes #3659 from JoshRosen/webui-help-sup and squashes the following commits:

bd72899 [Josh Rosen] Use <sup> tag for help icon in web UI page header.
2014-12-09 23:47:05 -08:00
Sandy Ryza 5e4c06f8e5 SPARK-4567. Make SparkJobInfo and SparkStageInfo serializable
Author: Sandy Ryza <sandy@cloudera.com>

Closes #3426 from sryza/sandy-spark-4567 and squashes the following commits:

cb4b8d2 [Sandy Ryza] SPARK-4567. Make SparkJobInfo and SparkStageInfo serializable
2014-12-09 16:26:07 -08:00
hushan[胡珊] 30dca924df [SPARK-4714] BlockManager.dropFromMemory() should check whether block has been removed after synchronizing on BlockInfo instance.
After synchronizing on the `info` lock in the `removeBlock`/`dropOldBlocks`/`dropFromMemory` methods in BlockManager, the block that `info` represented may have already removed.

The three methods have the same logic to get the `info` lock:
```
   info = blockInfo.get(id)
   if (info != null) {
     info.synchronized {
       // do something
     }
   }
```

So, there is chance that when a thread enters the `info.synchronized` block, `info` has already been removed from the `blockInfo` map by some other thread who entered `info.synchronized` first.

The `removeBlock` and `dropOldBlocks` methods are idempotent, so it's safe for them to run on blocks that have already been removed.
But in `dropFromMemory` it may be problematic since it may drop block data which already removed into the diskstore, and this calls data store operations that are not designed to handle missing blocks.

This patch fixes this issue by adding a check to `dropFromMemory` to test whether blocks have been removed by a racing thread.

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

Closes #3574 from suyanNone/refine-block-concurrency and squashes the following commits:

edb989d [hushan[胡珊]] Refine code style and comments position
55fa4ba [hushan[胡珊]] refine code
e57e270 [hushan[胡珊]] add check info is already remove or not while having gotten info.syn
2014-12-09 15:54:40 -08:00
Kay Ousterhout 1f5110630c [SPARK-4765] Make GC time always shown in UI.
This commit removes the GC time for each task from the set of
optional, additional metrics, and instead always shows it for
each task.

cc pwendell

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #3622 from kayousterhout/gc_time and squashes the following commits:

15ac242 [Kay Ousterhout] Make TaskDetailsClassNames private[spark]
e71d893 [Kay Ousterhout] [SPARK-4765] Make GC time always shown in UI.
2014-12-09 15:10:36 -08:00
maji2014 b31074466a [SPARK-4691][shuffle] Restructure a few lines in shuffle code
In HashShuffleReader.scala and HashShuffleWriter.scala, no need to judge "dep.aggregator.isEmpty" again as this is judged by "dep.aggregator.isDefined"

In SortShuffleWriter.scala, "dep.aggregator.isEmpty"  is better than "!dep.aggregator.isDefined" ?

Author: maji2014 <maji3@asiainfo.com>

Closes #3553 from maji2014/spark-4691 and squashes the following commits:

bf7b14d [maji2014] change a elegant way for SortShuffleWriter.scala
10d0cf0 [maji2014] change a elegant way
d8f52dc [maji2014] code optimization for judgement
2014-12-09 13:13:12 -08:00
Sean Owen e829bfa1ab SPARK-3926 [CORE] Reopened: result of JavaRDD collectAsMap() is not serializable
My original 'fix' didn't fix at all. Now, there's a unit test to check whether it works. Of the two options to really fix it -- copy the `Map` to a `java.util.HashMap`, or copy and modify Scala's implementation in `Wrappers.MapWrapper`, I went with the latter.

Author: Sean Owen <sowen@cloudera.com>

Closes #3587 from srowen/SPARK-3926 and squashes the following commits:

8586bb9 [Sean Owen] Remove unneeded no-arg constructor, and add additional note about copied code in LICENSE
7bb0e66 [Sean Owen] Make SerializableMapWrapper actually serialize, and add unit test
2014-12-08 16:13:03 -08:00
Andrew Or 65f929d5b3 [SPARK-4750] Dynamic allocation - synchronize kills
Simple omission on my part.

Author: Andrew Or <andrew@databricks.com>

Closes #3612 from andrewor14/dynamic-allocation-synchronization and squashes the following commits:

1f03b60 [Andrew Or] Synchronize kills
2014-12-08 16:02:33 -08:00
Christophe Préaud ab2abcb5ef [SPARK-4764] Ensure that files are fetched atomically
tempFile is created in the same directory than targetFile, so that the
move from tempFile to targetFile is always atomic

Author: Christophe Préaud <christophe.preaud@kelkoo.com>

Closes #2855 from preaudc/master and squashes the following commits:

9ba89ca [Christophe Préaud] Ensure that files are fetched atomically
54419ae [Christophe Préaud] Merge remote-tracking branch 'upstream/master'
c6a5590 [Christophe Préaud] Revert commit 8ea871f8130b2490f1bad7374a819bf56f0ccbbd
7456a33 [Christophe Préaud] Merge remote-tracking branch 'upstream/master'
8ea871f [Christophe Préaud] Ensure that files are fetched atomically
2014-12-08 11:44:54 -08:00
Zhang, Liye 98a7d09978 [SPARK-4005][CORE] handle message replies in receive instead of in the individual private methods
In BlockManagermasterActor, when handling message type UpdateBlockInfo, the message replies is in handled in individual private methods, should handle it in receive of Akka.

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

Closes #2853 from liyezhang556520/akkaRecv and squashes the following commits:

9b06f0a [Zhang, Liye] remove the unreachable code
bf518cd [Zhang, Liye] change the indent
242166b [Zhang, Liye] modified accroding to the comments
d4b929b [Zhang, Liye] [SPARK-4005][CORE] handle message replies in receive instead of in the individual private methods
2014-12-05 12:00:32 -08:00
Reynold Xin ed92b47e83 [SPARK-4397] Move object RDD to the front of RDD.scala.
I ran into multiple cases that SBT/Scala compiler was confused by the implicits in continuous compilation mode. Adding explicit return types fixes the problem.

Author: Reynold Xin <rxin@databricks.com>

Closes #3580 from rxin/rdd-implicit and squashes the following commits:

ee32fcd [Reynold Xin] Move object RDD to the end of the file.
b8562c9 [Reynold Xin] Merge branch 'master' of github.com:apache/spark into rdd-implicit
d4e9f85 [Reynold Xin] Code review.
a836a37 [Reynold Xin] Move object RDD to the front of RDD.scala.
2014-12-04 16:32:20 -08:00
Saldanha 743a889d27 [SPARK-4459] Change groupBy type parameter from K to U
Please see https://issues.apache.org/jira/browse/SPARK-4459

Author: Saldanha <saldaal1@phusca-l24858.wlan.na.novartis.net>

Closes #3327 from alokito/master and squashes the following commits:

54b1095 [Saldanha] [SPARK-4459] changed type parameter for keyBy from K to U
d5f73c3 [Saldanha] [SPARK-4459] added keyBy test
316ad77 [Saldanha] SPARK-4459 changed type parameter for groupBy from K to U.
62ddd4b [Saldanha] SPARK-4459 added failing unit test
2014-12-04 14:57:41 -08:00
WangTaoTheTonic 8106b1e36b [SPARK-4253] Ignore spark.driver.host in yarn-cluster and standalone-cluster modes
In yarn-cluster and standalone-cluster modes, we don't know where driver will run until it is launched.  If the `spark.driver.host` property is set on the submitting machine and propagated to the driver through SparkConf then this will lead to errors when the driver launches.

This patch fixes this issue by dropping the `spark.driver.host` property in SparkSubmit when running in a cluster deploy mode.

Author: WangTaoTheTonic <barneystinson@aliyun.com>
Author: WangTao <barneystinson@aliyun.com>

Closes #3112 from WangTaoTheTonic/SPARK4253 and squashes the following commits:

ed1a25c [WangTaoTheTonic] revert unrelated formatting issue
02c4e49 [WangTao] add comment
32a3f3f [WangTaoTheTonic] ingore it in SparkSubmit instead of SparkContext
667cf24 [WangTaoTheTonic] document fix
ff8d5f7 [WangTaoTheTonic] also ignore it in standalone cluster mode
2286e6b [WangTao] ignore spark.driver.host in yarn-cluster mode
2014-12-04 11:53:23 -08:00
Reynold Xin c3ad486036 [SPARK-4719][API] Consolidate various narrow dep RDD classes with MapPartitionsRDD
MappedRDD, MappedValuesRDD, FlatMappedValuesRDD, FilteredRDD, GlommedRDD, FlatMappedRDD are not necessary. They can be implemented trivially using MapPartitionsRDD.

Author: Reynold Xin <rxin@databricks.com>

Closes #3578 from rxin/SPARK-4719 and squashes the following commits:

eed9853 [Reynold Xin] Preserve partitioning for filter.
eb1a89b [Reynold Xin] [SPARK-4719][API] Consolidate various narrow dep RDD classes with MapPartitionsRDD.
2014-12-04 00:45:57 -08:00
Reynold Xin 1826372d0a [SPARK-4085] Propagate FetchFailedException when Spark fails to read local shuffle file.
cc aarondav kayousterhout pwendell

This should go into 1.2?

Author: Reynold Xin <rxin@databricks.com>

Closes #3579 from rxin/SPARK-4085 and squashes the following commits:

255b4fd [Reynold Xin] Updated test.
f9814d9 [Reynold Xin] Code review feedback.
2afaf35 [Reynold Xin] [SPARK-4085] Propagate FetchFailedException when Spark fails to read local shuffle file.
2014-12-03 16:28:24 -08:00
Mark Hamstra 96b27855c5 [SPARK-4498][core] Don't transition ExecutorInfo to RUNNING until Driver adds Executor
The ExecutorInfo only reaches the RUNNING state if the Driver is alive to send the ExecutorStateChanged message to master.  Else, appInfo.resetRetryCount() is never called and failing Executors will eventually exceed ApplicationState.MAX_NUM_RETRY, resulting in the application being removed from the master's accounting.

JoshRosen

Author: Mark Hamstra <markhamstra@gmail.com>

Closes #3550 from markhamstra/SPARK-4498 and squashes the following commits:

8f543b1 [Mark Hamstra] Don't transition ExecutorInfo to RUNNING until Executor is added by Driver
2014-12-03 15:08:01 -08:00
zsxwing edd3cd477c [SPARK-4715][Core] Make sure tryToAcquire won't return a negative value
ShuffleMemoryManager.tryToAcquire may return a negative value. The unit test demonstrates this bug. It will output `0 did not equal -200 granted is negative`.

Author: zsxwing <zsxwing@gmail.com>

Closes #3575 from zsxwing/SPARK-4715 and squashes the following commits:

a193ae6 [zsxwing] Make sure tryToAcquire won't return a negative value
2014-12-03 12:19:40 -08:00
zsxwing 8af551f71d [SPARK-4397][Core] Change the 'since' value of '@deprecated' to '1.3.0'
As #3262 wasn't merged to branch 1.2, the `since` value of `deprecated` should be '1.3.0'.

Author: zsxwing <zsxwing@gmail.com>

Closes #3573 from zsxwing/SPARK-4397-version and squashes the following commits:

1daa03c [zsxwing] Change the 'since' value to '1.3.0'
2014-12-03 02:05:17 -08:00
JerryLead 77be8b986f [SPARK-4672][Core]Checkpoint() should clear f to shorten the serialization chain
The related JIRA is https://issues.apache.org/jira/browse/SPARK-4672

The f closure of `PartitionsRDD(ZippedPartitionsRDD2)` contains a `$outer` that references EdgeRDD/VertexRDD, which causes task's serialization chain become very long in iterative GraphX applications. As a result, StackOverflow error will occur. If we set "f = null" in `clearDependencies()`, checkpoint() can cut off the long serialization chain. More details and explanation can be found in the JIRA.

Author: JerryLead <JerryLead@163.com>
Author: Lijie Xu <csxulijie@gmail.com>

Closes #3545 from JerryLead/my_core and squashes the following commits:

f7faea5 [JerryLead] checkpoint() should clear the f to avoid StackOverflow error
c0169da [JerryLead] Merge branch 'master' of https://github.com/apache/spark
52799e3 [Lijie Xu] Merge pull request #1 from apache/master
2014-12-02 23:53:29 -08:00
zsxwing 6dfe38a03a [SPARK-4397][Core] Cleanup 'import SparkContext._' in core
This PR cleans up `import SparkContext._` in core for SPARK-4397(#3262) to prove it really works well.

Author: zsxwing <zsxwing@gmail.com>

Closes #3530 from zsxwing/SPARK-4397-cleanup and squashes the following commits:

04e2273 [zsxwing] Cleanup 'import SparkContext._' in core
2014-12-02 00:18:41 -08:00
zsxwing 30a86acdef [SPARK-4661][Core] Minor code and docs cleanup
Author: zsxwing <zsxwing@gmail.com>

Closes #3521 from zsxwing/SPARK-4661 and squashes the following commits:

03cbe3f [zsxwing] Minor code and docs cleanup
2014-12-01 00:35:01 -08:00
zsxwing 1d238f221c [SPARK-4664][Core] Throw an exception when spark.akka.frameSize > 2047
If `spark.akka.frameSize` > 2047, it will overflow and become negative. Should have some assertion in `maxFrameSizeBytes` to warn people.

Author: zsxwing <zsxwing@gmail.com>

Closes #3527 from zsxwing/SPARK-4664 and squashes the following commits:

0089c7a [zsxwing] Throw an exception when spark.akka.frameSize > 2047
2014-12-01 00:32:54 -08:00
carlmartin aea7a99761 [SPARK-4623]Add the some error infomation if using spark-sql in yarn-cluster mode
If using spark-sql in yarn-cluster mode, print an error infomation just as the spark shell in yarn-cluster mode.

Author: carlmartin <carlmartinmax@gmail.com>
Author: huangzhaowei <carlmartinmax@gmail.com>

Closes #3479 from SaintBacchus/sparkSqlShell and squashes the following commits:

35829a9 [carlmartin] improve the description of comment
e6c1eb7 [carlmartin] add a comment in bin/spark-sql to remind user who wants to change the class
f1c5c8d [carlmartin] Merge branch 'master' into sparkSqlShell
8e112c5 [huangzhaowei] singular form
ec957bc [carlmartin] Add the some error infomation if using spark-sql in yarn-cluster mode
7bcecc2 [carlmartin] Merge branch 'master' of https://github.com/apache/spark into codereview
4fad75a [carlmartin] Add the Error infomation using spark-sql in yarn-cluster mode
2014-11-30 16:19:41 -08:00
Sean Owen 048ecca625 SPARK-2143 [WEB UI] Add Spark version to UI footer
This PR adds the Spark version number to the UI footer; this is how it looks:

![screen shot 2014-11-21 at 22 58 40](https://cloud.githubusercontent.com/assets/822522/5157738/f4822094-7316-11e4-98f1-333a535fdcfa.png)

Author: Sean Owen <sowen@cloudera.com>

Closes #3410 from srowen/SPARK-2143 and squashes the following commits:

e9b3a7a [Sean Owen] Add Spark version to footer
2014-11-30 11:41:38 -08:00
zsxwing c06222427f [SPARK-4505][Core] Add a ClassTag parameter to CompactBuffer[T]
Added a ClassTag parameter to CompactBuffer. So CompactBuffer[T] can create primitive arrays for primitive types. It will reduce the memory usage for primitive types significantly and only pay minor performance lost.

Here is my test code:
```Scala
  // Call org.apache.spark.util.SizeEstimator.estimate
  def estimateSize(obj: AnyRef): Long = {
    val c = Class.forName("org.apache.spark.util.SizeEstimator$")
    val f = c.getField("MODULE$")
    val o = f.get(c)
    val m = c.getMethod("estimate", classOf[Object])
    m.setAccessible(true)
    m.invoke(o, obj).asInstanceOf[Long]
  }

  sc.parallelize(1 to 10000).groupBy(_ => 1).foreach {
    case (k, v) =>
      println(v.getClass() + " size: " + estimateSize(v))
  }
```

Using the previous CompactBuffer outputed
```
class org.apache.spark.util.collection.CompactBuffer size: 313358
```

Using the new CompactBuffer outputed
```
class org.apache.spark.util.collection.CompactBuffer size: 65712
```

In this case, the new `CompactBuffer` only used 20% memory of the previous one. It's really helpful for `groupByKey` when using a primitive value.

Author: zsxwing <zsxwing@gmail.com>

Closes #3378 from zsxwing/SPARK-4505 and squashes the following commits:

4abdbba [zsxwing] Add a ClassTag parameter to reduce the memory usage of CompactBuffer[T] when T is a primitive type
2014-11-29 20:23:08 -05:00
Stephen Haberman 95290bf4c4 Include the key name when failing on an invalid value.
Admittedly a really small tweak.

Author: Stephen Haberman <stephen@exigencecorp.com>

Closes #3514 from stephenh/include-key-name-in-npe and squashes the following commits:

937740a [Stephen Haberman] Include the key name when failing on an invalid value.
2014-11-29 20:12:05 -05:00
Liang-Chi Hsieh 49fe8797e6 [SPARK-4597] Use proper exception and reset variable in Utils.createTempDir()
`File.exists()` and `File.mkdirs()` only throw `SecurityException` instead of `IOException`. Then, when an exception is thrown, `dir` should be reset too.

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

Closes #3449 from viirya/fix_createtempdir and squashes the following commits:

36cacbd [Liang-Chi Hsieh] Use proper exception and reset variable.
2014-11-28 18:04:15 -08:00
maji2014 ceb6281970 [SPARK-4619][Storage]delete redundant time suffix
Time suffix exists in Utils.getUsedTimeMs(startTime), no need to append again, delete that

Author: maji2014 <maji3@asiainfo.com>

Closes #3475 from maji2014/SPARK-4619 and squashes the following commits:

df0da4e [maji2014] delete redundant time suffix
2014-11-28 00:36:22 -08:00
Cheng Lian 120a350240 [SPARK-4613][Core] Java API for JdbcRDD
This PR introduces a set of Java APIs for using `JdbcRDD`:

1. Trait (interface) `JdbcRDD.ConnectionFactory`: equivalent to the `getConnection: () => Connection` parameter in `JdbcRDD` constructor.
2. Two overloaded versions of `Jdbc.create`: used to create `JavaRDD` that wraps a `JdbcRDD`.

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

Author: Cheng Lian <lian@databricks.com>

Closes #3478 from liancheng/japi-jdbc-rdd and squashes the following commits:

9a54625 [Cheng Lian] Only shutdowns a single DB rather than the whole Derby driver
d4cedc5 [Cheng Lian] Moves Java JdbcRDD test case to a separate test suite
ffcdf2e [Cheng Lian] Java API for JdbcRDD
2014-11-27 18:01:14 -08:00
roxchkplusony 84376d3139 [SPARK-4626] Kill a task only if the executorId is (still) registered with the scheduler
Author: roxchkplusony <roxchkplusony@gmail.com>

Closes #3483 from roxchkplusony/bugfix/4626 and squashes the following commits:

aba9184 [roxchkplusony] replace warning message per review
5e7fdea [roxchkplusony] [SPARK-4626] Kill a task only if the executorId is (still) registered with the scheduler
2014-11-27 15:54:40 -08:00
Sean Owen 5d7fe178b3 SPARK-4170 [CORE] Closure problems when running Scala app that "extends App"
Warn against subclassing scala.App, and remove one instance of this in examples

Author: Sean Owen <sowen@cloudera.com>

Closes #3497 from srowen/SPARK-4170 and squashes the following commits:

4a6131f [Sean Owen] Restore multiline string formatting
a8ca895 [Sean Owen] Warn against subclassing scala.App, and remove one instance of this in examples
2014-11-27 09:03:17 -08:00
CodingCat 5af53ada65 [SPARK-732][SPARK-3628][CORE][RESUBMIT] eliminate duplicate update on accmulator
https://issues.apache.org/jira/browse/SPARK-3628

In current implementation, the accumulator will be updated for every successfully finished task, even the task is from a resubmitted stage, which makes the accumulator counter-intuitive

In this patch, I changed the way for the DAGScheduler to update the accumulator,

DAGScheduler maintains a HashTable, mapping the stage id to the received <accumulator_id , value> pairs. Only when the stage becomes independent, (no job needs it any more), we accumulate the values of the <accumulator_id , value> pairs, when a task finished, we check if the HashTable has contained such stageId, it saves the accumulator_id, value only when the task is the first finished task of a new stage or the stage is running for the first attempt...

Author: CodingCat <zhunansjtu@gmail.com>

Closes #2524 from CodingCat/SPARK-732-1 and squashes the following commits:

701a1e8 [CodingCat] roll back change on Accumulator.scala
1433e6f [CodingCat] make MIMA happy
b233737 [CodingCat] address Matei's comments
02261b8 [CodingCat] rollback  some changes
6b0aff9 [CodingCat] update document
2b2e8cf [CodingCat] updateAccumulator
83b75f8 [CodingCat] style fix
84570d2 [CodingCat] re-enable  the bad accumulator guard
1e9e14d [CodingCat] add NPE guard
21b6840 [CodingCat] simplify the patch
88d1f03 [CodingCat] fix rebase error
f74266b [CodingCat] add test case for resubmitted result stage
5cf586f [CodingCat] de-duplicate on task level
138f9b3 [CodingCat] make MIMA happy
67593d2 [CodingCat] make if allowing duplicate update as an option of accumulator
2014-11-26 16:52:04 -08:00
Tathagata Das e7f4d2534b [SPARK-4612] Reduce task latency and increase scheduling throughput by making configuration initialization lazy
https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/executor/Executor.scala#L337 creates a configuration object for every task that is launched, even if there is no new dependent file/JAR to update. This is a heavy-weight creation that should be avoided if there is no new file/JAR to update. This PR makes that creation lazy. Quick local test in spark-perf scheduling throughput tests gives the following numbers in a local standalone scheduler mode.
1 job with 10000 tasks: before 7.8395 seconds, after 2.6415 seconds = 3x increase in task scheduling throughput

pwendell JoshRosen

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

Closes #3463 from tdas/lazy-config and squashes the following commits:

c791c1e [Tathagata Das] Reduce task latency by making configuration initialization lazy
2014-11-25 23:15:58 -08:00
Aaron Davidson f5f2d27385 [SPARK-4516] Cap default number of Netty threads at 8
In practice, only 2-4 cores should be required to transfer roughly 10 Gb/s, and each core that we use will have an initial overhead of roughly 32 MB of off-heap memory, which comes at a premium.

Thus, this value should still retain maximum throughput and reduce wasted off-heap memory allocation. It can be overridden by setting the number of serverThreads and clientThreads manually in Spark's configuration.

Author: Aaron Davidson <aaron@databricks.com>

Closes #3469 from aarondav/fewer-pools2 and squashes the following commits:

087c59f [Aaron Davidson] [SPARK-4516] Cap default number of Netty threads at 8
2014-11-25 23:57:04 -05:00
hushan[胡珊] 9bdf5da590 Fix SPARK-4471: blockManagerIdFromJson function throws exception while B...
Fix [SPARK-4471](https://issues.apache.org/jira/browse/SPARK-4471): blockManagerIdFromJson function throws exception while BlockManagerId be null in MetadataFetchFailedException

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

Closes #3340 from suyanNone/fix-blockmanagerId-jnothing-2 and squashes the following commits:

159f9a3 [hushan[胡珊]] Refine test code for blockmanager is null
4380d73 [hushan[胡珊]] remove useless blank line
3ccf651 [hushan[胡珊]] Fix SPARK-4471: blockManagerIdFromJson function throws exception while metadata fetch failed
2014-11-25 15:51:08 -08:00
Andrew Or 9afcbe494a [SPARK-4546] Improve HistoryServer first time user experience
The documentation points the user to run the following
```
sbin/start-history-server.sh
```
The first thing this does is throw an exception that complains a log directory is not specified. The exception message itself does not say anything about what to set. Instead we should have a default and a landing page with a better message. The new default log directory is `file:/tmp/spark-events`.

This is what it looks like as of this PR:

![after](https://issues.apache.org/jira/secure/attachment/12682985/after.png)

Author: Andrew Or <andrew@databricks.com>

Closes #3411 from andrewor14/minor-history-improvements and squashes the following commits:

f33d6b3 [Andrew Or] Point user to set config if default log dir does not exist
fc4c17a [Andrew Or] Improve HistoryServer UX
2014-11-25 15:48:02 -08:00
Andrew Or 1b2ab1cd1b [SPARK-4592] Avoid duplicate worker registrations in standalone mode
**Summary.** On failover, the Master may receive duplicate registrations from the same worker, causing the worker to exit. This is caused by this commit 4afe9a4852, which adds logic for the worker to re-register with the master in case of failures. However, the following race condition may occur:

(1) Master A fails and Worker attempts to reconnect to all masters
(2) Master B takes over and notifies Worker
(3) Worker responds by registering with Master B
(4) Meanwhile, Worker's previous reconnection attempt reaches Master B, causing the same Worker to register with Master B twice

**Fix.** Instead of attempting to register with all known masters, the worker should re-register with only the one that it has been communicating with. This is safe because the fact that a failover has occurred means the old master must have died. Then, when the worker is finally notified of a new master, it gives up on the old one in favor of the new one.

**Caveat.** Even this fix is subject to more obscure race conditions. For instance, if Master B fails and Master A recovers immediately, then Master A may still observe duplicate worker registrations. However, this and other potential race conditions summarized in [SPARK-4592](https://issues.apache.org/jira/browse/SPARK-4592), are much, much less likely than the one described above, which is deterministically reproducible.

Author: Andrew Or <andrew@databricks.com>

Closes #3447 from andrewor14/standalone-failover and squashes the following commits:

0d9716c [Andrew Or] Move re-registration logic to actor for thread-safety
79286dc [Andrew Or] Preserve old behavior for initial retries
83b321c [Andrew Or] Tweak wording
1fce6a9 [Andrew Or] Active master actor could be null in the beginning
b6f269e [Andrew Or] Avoid duplicate worker registrations
2014-11-25 15:46:26 -08:00
Jongyoul Lee f0afb623dc [SPARK-4525] Mesos should decline unused offers
Functionally, this is just a small change on top of #3393 (by jongyoul). The issue being addressed is discussed in the comments there. I have not yet added a test for the bug there. I will add one shortly.

I've also done some minor renaming/clean-up of variables in this class and tests.

Author: Patrick Wendell <pwendell@gmail.com>
Author: Jongyoul Lee <jongyoul@gmail.com>

Closes #3436 from pwendell/mesos-issue and squashes the following commits:

58c35b5 [Patrick Wendell] Adding unit test for this situation
c4f0697 [Patrick Wendell] Additional clean-up and fixes on top of existing fix
f20f1b3 [Jongyoul Lee] [SPARK-4525] MesosSchedulerBackend.resourceOffers cannot decline unused offers from acceptedOffers - Added code for declining unused offers among acceptedOffers - Edited testCase for checking declining unused offers
2014-11-24 19:19:16 -08:00
Patrick Wendell a68d442270 Revert "[SPARK-4525] Mesos should decline unused offers"
This reverts commit b043c27424.

I accidentally committed this using my own authorship credential. However,
I should have given authoriship to the original author: Jongyoul Lee.
2014-11-24 19:18:15 -08:00
Patrick Wendell b043c27424 [SPARK-4525] Mesos should decline unused offers
Functionally, this is just a small change on top of #3393 (by jongyoul). The issue being addressed is discussed in the comments there. I have not yet added a test for the bug there. I will add one shortly.

I've also done some minor renaming/clean-up of variables in this class and tests.

Author: Patrick Wendell <pwendell@gmail.com>
Author: Jongyoul Lee <jongyoul@gmail.com>

Closes #3436 from pwendell/mesos-issue and squashes the following commits:

58c35b5 [Patrick Wendell] Adding unit test for this situation
c4f0697 [Patrick Wendell] Additional clean-up and fixes on top of existing fix
f20f1b3 [Jongyoul Lee] [SPARK-4525] MesosSchedulerBackend.resourceOffers cannot decline unused offers from acceptedOffers - Added code for declining unused offers among acceptedOffers - Edited testCase for checking declining unused offers
2014-11-24 19:14:14 -08:00
Kay Ousterhout d24d5bf064 [SPARK-4266] [Web-UI] Reduce stage page load time.
The commit changes the java script used to show/hide additional
metrics in order to reduce page load time. SPARK-4016 significantly
increased page load time for the stage page when stages had a lot
(thousands or tens of thousands) of tasks, due to the additional
Javascript to hide some metrics by default and stripe the tables.
This commit reduces page load time in two ways:

(1) Now, all of the metrics that are hidden by default are
hidden by setting "display: none;" using CSS for the page,
rather than hiding them using javascript after the page loads.
Without this change, for stages with thousands of tasks, there
was a few second delay after page load, where first the additional
metrics were shown, and then after a delay were hidden once the
relevant JS finished running.

(2) CSS is used to stripe all of the tables except for the summary
table. The summary table needs javascript to do the striping because
some rows are hidden, but the javascript striping is slower, which
again resulted in a delay when it was used for the task table (where
for a few seconds after page load, all of the rows in the task table
would be white, while the browser finished running the JS to stripe
the table).

cc pwendell

This change is intended to be backported to 1.2 to avoid a regression in
UI performance when users run large jobs.

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #3328 from kayousterhout/SPARK-4266 and squashes the following commits:

f964091 [Kay Ousterhout] [SPARK-4266] [Web-UI] Reduce stage page load time.
2014-11-24 18:03:10 -08:00
Davies Liu 6cf507685e [SPARK-4548] []SPARK-4517] improve performance of python broadcast
Re-implement the Python broadcast using file:

1) serialize the python object using cPickle, write into disks.
2) Create a wrapper in JVM (for the dumped file), it read data from during serialization
3) Using TorrentBroadcast or HttpBroadcast to transfer the data (compressed) into executors
4) During deserialization, writing the data into disk.
5) Passing the path into Python worker, read data from disk and unpickle it into python object, until the first access.

It fixes the performance regression introduced in #2659, has similar performance as 1.1, but support object larger than 2G, also improve the memory efficiency (only one compressed copy in driver and executor).

Testing with a 500M broadcast and 4 tasks (excluding the benefit from reused worker in 1.2):

         name |   1.1   | 1.2 with this patch |  improvement
---------|--------|---------|--------
      python-broadcast-w-bytes  |	25.20  |	9.33   |	170.13% |
        python-broadcast-w-set	  |     4.13	   |    4.50  |	-8.35%  |

Testing with 100 tasks (16 CPUs):

         name |   1.1   | 1.2 with this patch |  improvement
---------|--------|---------|--------
     python-broadcast-w-bytes	| 38.16	| 8.40	 | 353.98%
        python-broadcast-w-set	| 23.29	| 9.59 |	142.80%

Author: Davies Liu <davies@databricks.com>

Closes #3417 from davies/pybroadcast and squashes the following commits:

50a58e0 [Davies Liu] address comments
b98de1d [Davies Liu] disable gc while unpickle
e5ee6b9 [Davies Liu] support large string
09303b8 [Davies Liu] read all data into memory
dde02dd [Davies Liu] improve performance of python broadcast
2014-11-24 17:17:03 -08:00
Josh Rosen 4a90276ab2 [SPARK-4145] Web UI job pages
This PR adds two new pages to the Spark Web UI:

- A jobs overview page, which shows details on running / completed / failed jobs.
- A job details page, which displays information on an individual job's stages.

The jobs overview page is now the default UI homepage; the old homepage is still accessible at `/stages`.

### Screenshots

#### New UI homepage

![image](https://cloud.githubusercontent.com/assets/50748/5119035/fd0a69e6-701f-11e4-89cb-db7e9705714f.png)

#### Job details page

(This is effectively a per-job version of the stages page that can be extended later with other things, such as DAG visualizations)

![image](https://cloud.githubusercontent.com/assets/50748/5134910/50b340d4-70c7-11e4-88e1-6b73237ea7c8.png)

### Key changes in this PR

- Rename `JobProgressPage` to `AllStagesPage`
- Expose `StageInfo` objects in the ``SparkListenerJobStart` event; add backwards-compatibility tests to JsonProtocol.
- Add additional data structures to `JobProgressListener` to map from stages to jobs.
- Add several fields to `JobUIData`.

I also added ~150 lines of Selenium tests as I uncovered UI issues while developing this patch.

### Limitations

If a job contains stages that aren't run, then its overall job progress bar may be an underestimate of the total job progress; in other words, a completed job may appear to have a progress bar that's not at 100%.

If stages or tasks fail, then the progress bar will not go backwards to reflect the true amount of remaining work.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #3009 from JoshRosen/job-page and squashes the following commits:

eb05e90 [Josh Rosen] Disable kill button in completed stages tables.
f00c851 [Josh Rosen] Fix JsonProtocol compatibility
b89c258 [Josh Rosen] More JSON protocol backwards-compatibility fixes.
ff804cd [Josh Rosen] Don't write "Stage Ids" field in JobStartEvent JSON.
6f17f3f [Josh Rosen] Only store StageInfos in SparkListenerJobStart event.
2bbf41a [Josh Rosen] Update job progress bar to reflect skipped tasks/stages.
61c265a [Josh Rosen] Add “skipped stages” table; only display non-empty tables.
1f45d44 [Josh Rosen] Incorporate a bunch of minor review feedback.
0b77e3e [Josh Rosen] More bug fixes for phantom stages.
034aa8d [Josh Rosen] Use `.max()` to find result stage for job.
eebdc2c [Josh Rosen] Don’t display pending stages for completed jobs.
67080ba [Josh Rosen] Ensure that "phantom stages" don't cause memory leaks.
7d10b97 [Josh Rosen] Merge remote-tracking branch 'apache/master' into job-page
d69c775 [Josh Rosen] Fix table sorting on all jobs page.
5eb39dc [Josh Rosen] Add pending stages table to job page.
f2a15da [Josh Rosen] Add status field to job details page.
171b53c [Josh Rosen] Move `startTime` to the start of SparkContext.
e2f2c43 [Josh Rosen] Fix sorting of stages in job details page.
8955f4c [Josh Rosen] Display information for pending stages on jobs page.
8ab6c28 [Josh Rosen] Compute numTasks from job start stage infos.
5884f91 [Josh Rosen] Add StageInfos to SparkListenerJobStart event.
79793cd [Josh Rosen] Track indices of completed stage to avoid overcounting when failures occur.
d62ea7b [Josh Rosen] Add failing Selenium test for stage overcounting issue.
1145c60 [Josh Rosen] Display text instead of progress bar for stages.
3d0a007 [Josh Rosen] Merge remote-tracking branch 'origin/master' into job-page
8a2351b [Josh Rosen] Add help tooltip to Spark Jobs page.
b7bf30e [Josh Rosen] Add stages progress bar; fix bug where active stages show as completed.
4846ce4 [Josh Rosen] Hide "(Job Group") if no jobs were submitted in job groups.
4d58e55 [Josh Rosen] Change label to "Tasks (for all stages)"
85e9c85 [Josh Rosen] Extract startTime into separate variable.
1cf4987 [Josh Rosen] Fix broken kill links; add Selenium test to avoid future regressions.
56701fa [Josh Rosen] Move last stage name / description logic out of markup.
a475ea1 [Josh Rosen] Add progress bars to jobs page.
45343b8 [Josh Rosen] More comments
4b206fb [Josh Rosen] Merge remote-tracking branch 'origin/master' into job-page
bfce2b9 [Josh Rosen] Address review comments, except for progress bar.
4487dcb [Josh Rosen] [SPARK-4145] Web UI job pages
2568a6c [Josh Rosen] Rename JobProgressPage to AllStagesPage:
2014-11-24 13:18:14 -08:00
Cheng Lian a6d7b61f92 [SPARK-4479][SQL] Avoids unnecessary defensive copies when sort based shuffle is on
This PR is a workaround for SPARK-4479. Two changes are introduced: when merge sort is bypassed in `ExternalSorter`,

1. also bypass RDD elements buffering as buffering is the reason that `MutableRow` backed row objects must be copied, and
2. avoids defensive copies in `Exchange` operator

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

Author: Cheng Lian <lian@databricks.com>

Closes #3422 from liancheng/avoids-defensive-copies and squashes the following commits:

591f2e9 [Cheng Lian] Passes all shuffle suites
0c3c91e [Cheng Lian] Fixes shuffle write metrics when merge sort is bypassed
ed5df3c [Cheng Lian] Fixes styling changes
f75089b [Cheng Lian] Avoids unnecessary defensive copies when sort based shuffle is on
2014-11-24 12:43:45 -08:00
Prashant Sharma 9b2a3c6126 [SPARK-4377] Fixed serialization issue by switching to akka provided serializer.
... - there is no way around this for deserializing actorRef(s).

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

Closes #3402 from ScrapCodes/SPARK-4377/troubleDeserializing and squashes the following commits:

77233fd [Prashant Sharma] Style fixes
9b35c6e [Prashant Sharma] Scalastyle fixes
29880da [Prashant Sharma] [SPARK-4377] Fixed serialization issue by switching to akka provided serializer - there is no way around this for deserializing actorRef(s).
2014-11-22 14:05:38 -08:00
zsxwing 65b987c3ed [SPARK-4397][Core] Reorganize 'implicit's to improve the API convenience
This PR moved `implicit`s to `package object` and `companion object` to enable the Scala compiler search them automatically without explicit importing.

It should not break any API. A test project for backforward compatibility is [here](https://github.com/zsxwing/SPARK-4397-Backforward-Compatibility). It proves the codes compiled with Spark 1.1.0 can run with this PR.

To summarize, the changes are:

* Deprecated the old implicit conversion functions: this preserves binary compatibility for code compiled against earlier versions of Spark.
* Removed "implicit" from them so they are just normal functions: this made sure the compiler doesn't get confused and warn about multiple implicits in scope.
* Created new implicit functions in package rdd object, which is part of the scope that scalac will search when looking for implicit conversions on various RDD objects.

The disadvantage is there are duplicated codes in SparkContext for backforward compatibility.

Author: zsxwing <zsxwing@gmail.com>

Closes #3262 from zsxwing/SPARK-4397 and squashes the following commits:

fc30314 [zsxwing] Update the comments
9c27aff [zsxwing] Move implicit functions to object RDD and forward old functions to new implicit ones directly
2b5f5a4 [zsxwing] Comments for the deprecated functions
52353de [zsxwing] Remove private[spark] from object WritableConverter
34641d4 [zsxwing] Move ImplicitSuite to org.apache.sparktest
7266218 [zsxwing] Add comments to warn the duplicate codes in SparkContext
185c12f [zsxwing] Remove simpleWritableConverter from SparkContext
3bdcae2 [zsxwing] Move WritableConverter implicits to object WritableConverter
9b73188 [zsxwing] Fix the code style issue
3ac4f07 [zsxwing] Add license header
1eda9e4 [zsxwing] Reorganize 'implicit's to improve the API convenience
2014-11-21 10:06:30 -08:00
Leolh e216ffaead [SPARK-4446] [SPARK CORE]
MetadataCleaner schedule task with a wrong param for delay time .

Author: Leolh <leosandylh@gmail.com>

Closes #3306 from Leolh/master and squashes the following commits:

4a21f4e [Leolh] Update MetadataCleaner.scala
2014-11-19 18:18:55 -08:00
Andrew Or 0eb4a7fb0f [SPARK-4480] Avoid many small spills in external data structures
**Summary.** Currently, we may spill many small files in `ExternalAppendOnlyMap` and `ExternalSorter`. The underlying root cause of this is summarized in [SPARK-4452](https://issues.apache.org/jira/browse/SPARK-4452). This PR does not address this root cause, but simply provides the guarantee that we never spill the in-memory data structure if its size is less than a configurable threshold of 5MB. This config is not documented because we don't want users to set it themselves, and it is not hard-coded because we need to change it in tests.

**Symptom.** Each spill is orders of magnitude smaller than 1MB, and there are many spills. In environments where the ulimit is set, this frequently causes "too many open file" exceptions observed in [SPARK-3633](https://issues.apache.org/jira/browse/SPARK-3633).
```
14/11/13 19:20:43 INFO collection.ExternalSorter: Thread 60 spilling in-memory batch of 4792 B to disk (292769 spills so far)
14/11/13 19:20:43 INFO collection.ExternalSorter: Thread 60 spilling in-memory batch of 4760 B to disk (292770 spills so far)
14/11/13 19:20:43 INFO collection.ExternalSorter: Thread 60 spilling in-memory batch of 4520 B to disk (292771 spills so far)
14/11/13 19:20:43 INFO collection.ExternalSorter: Thread 60 spilling in-memory batch of 4560 B to disk (292772 spills so far)
14/11/13 19:20:43 INFO collection.ExternalSorter: Thread 60 spilling in-memory batch of 4792 B to disk (292773 spills so far)
14/11/13 19:20:43 INFO collection.ExternalSorter: Thread 60 spilling in-memory batch of 4784 B to disk (292774 spills so far)
```

**Reproduction.** I ran the following on a small 4-node cluster with 512MB executors. Note that the back-to-back shuffle here is necessary for reasons described in [SPARK-4522](https://issues.apache.org/jira/browse/SPARK-4452). The second shuffle is a `reduceByKey` because it performs a map-side combine.
```
sc.parallelize(1 to 100000000, 100)
  .map { i => (i, i) }
  .groupByKey()
  .reduceByKey(_ ++ _)
  .count()
```
Before the change, I notice that each thread may spill up to 1000 times, and the size of each spill is on the order of 10KB. After the change, each thread spills only up to 20 times in the worst case, and the size of each spill is on the order of 1MB.

Author: Andrew Or <andrew@databricks.com>

Closes #3353 from andrewor14/avoid-small-spills and squashes the following commits:

49f380f [Andrew Or] Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/spark into avoid-small-spills
27d6966 [Andrew Or] Merge branch 'master' of github.com:apache/spark into avoid-small-spills
f4736e3 [Andrew Or] Fix tests
a919776 [Andrew Or] Avoid many small spills
2014-11-19 18:07:27 -08:00
Nishkam Ravi 73fedf5a6e [Spark-4484] Treat maxResultSize as unlimited when set to 0; improve error message
The check for maxResultSize > 0 is missing, results in failures. Also, error message needs to be improved so the developers know that there is a new parameter to be configured

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

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

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
2014-11-19 17:23:42 -08:00
Akshat Aranya 9ccc53c72c [SPARK-4478] Keep totalRegisteredExecutors up-to-date
This rebases PR 3368.

This commit fixes totalRegisteredExecutors update [SPARK-4478], so that we can correctly keep track of number of registered executors.

Author: Akshat Aranya <aaranya@quantcast.com>

Closes #3373 from coolfrood/topic/SPARK-4478 and squashes the following commits:

8a4d1e4 [Akshat Aranya] Added comment
150ae93 [Akshat Aranya] [SPARK-4478] Keep totalRegisteredExecutors up-to-date
2014-11-19 17:20:20 -08:00
Josh Rosen 04d462f648 [SPARK-4495] Fix memory leak in JobProgressListener
This commit fixes a memory leak in JobProgressListener that I introduced in SPARK-2321 and adds a testing framework to ensure that it’s very difficult to inadvertently introduce new memory leaks.

This solution might be overkill, but the main idea is to partition JobProgressListener's state into three buckets: collections that should be empty once Spark is idle, collections that must obey some hard size limit, and collections that have a soft size limit (they can grow arbitrarily large when Spark is active but must shrink to fit within some bound after Spark becomes idle).

Based on this, we can write fairly generic tests that run workloads that submit more than `spark.ui.retainedStages` stages and `spark.ui.retainedJobs` jobs then check that these various collections' sizes obey their contracts.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #3372 from JoshRosen/SPARK-4495 and squashes the following commits:

c73fab5 [Josh Rosen] "data structures" -> collections
be72e81 [Josh Rosen] [SPARK-4495] Fix memory leaks in JobProgressListener
2014-11-19 16:50:21 -08:00
Kenichi Maehashi eacc788346 [SPARK-4470] Validate number of threads in local mode
When running Spark locally, if number of threads is specified as 0 (e.g., `spark-submit --master local[0] ...`), the job got stuck and does not run at all.
I think it's better to validate the parameter.

Fix for [SPARK-4470](https://issues.apache.org/jira/browse/SPARK-4470).

Author: Kenichi Maehashi <webmaster@kenichimaehashi.com>

Closes #3337 from kmaehashi/spark-4470 and squashes the following commits:

3ad76f3 [Kenichi Maehashi] fix code style
7716734 [Kenichi Maehashi] SPARK-4470: Validate number of threads in local mode
2014-11-19 12:11:09 -08:00
Tianshuo Deng d75579d099 [SPARK-4467] fix elements read count for ExtrenalSorter
the elementsRead variable should be reset to 0 after each spilling

Author: Tianshuo Deng <tdeng@twitter.com>

Closes #3302 from tsdeng/fix_external_sorter_record_count and squashes the following commits:

7b56ca0 [Tianshuo Deng] fix method signature
782c7de [Tianshuo Deng] make elementsRead private, fix comment
bb7ff28 [Tianshuo Deng] update elemetsRead through addElementsRead method
74ca246 [Tianshuo Deng] fix elements read count
2014-11-19 10:02:16 -08:00
Mingfei 165cec9c46 [Spark-4432]close InStream after the block is accessed
InStream is not closed after data is read from Tachyon. which makes the blocks in Tachyon locked after accessed.

Author: Mingfei <mingfei.shi@intel.com>

Closes #3290 from shimingfei/lockFix and squashes the following commits:

fffe345 [Mingfei] close InStream after the block is accessed
2014-11-18 22:17:06 -08:00
Mingfei 67e9876b3e [SPARK-4441] Close Tachyon client when TachyonBlockManager is shutdown
Currently Tachyon client is not closed when TachyonBlockManager is shut down. which causes some resources in Tachyon not reclaimed

Author: Mingfei <mingfei.shi@intel.com>

Closes #3299 from shimingfei/closeClient and squashes the following commits:

0913fbd [Mingfei] close Tachyon client when TachyonBlockManager is shutdown
2014-11-18 22:16:36 -08:00
Marcelo Vanzin 397d3aae5b Bumping version to 1.3.0-SNAPSHOT.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #3277 from vanzin/version-1.3 and squashes the following commits:

7c3c396 [Marcelo Vanzin] Added temp repo to sbt build.
5f404ff [Marcelo Vanzin] Add another exclusion.
19457e7 [Marcelo Vanzin] Update old version to 1.2, add temporary 1.2 repo.
3c8d705 [Marcelo Vanzin] Workaround for MIMA checks.
e940810 [Marcelo Vanzin] Bumping version to 1.3.0-SNAPSHOT.
2014-11-18 21:24:18 -08:00
Xiangrui Meng bb46046154 [SPARK-4433] fix a racing condition in zipWithIndex
Spark hangs with the following code:

~~~
sc.parallelize(1 to 10).zipWithIndex.repartition(10).count()
~~~

This is because ZippedWithIndexRDD triggers a job in getPartitions and it causes a deadlock in DAGScheduler.getPreferredLocs (synced). The fix is to compute `startIndices` during construction.

This should be applied to branch-1.0, branch-1.1, and branch-1.2.

pwendell

Author: Xiangrui Meng <meng@databricks.com>

Closes #3291 from mengxr/SPARK-4433 and squashes the following commits:

c284d9f [Xiangrui Meng] fix a racing condition in zipWithIndex
2014-11-18 16:25:44 -08:00
Davies Liu 4a377aff2d [SPARK-3721] [PySpark] broadcast objects larger than 2G
This patch will bring support for broadcasting objects larger than 2G.

pickle, zlib, FrameSerializer and Array[Byte] all can not support objects larger than 2G, so this patch introduce LargeObjectSerializer to serialize broadcast objects, the object will be serialized and compressed into small chunks, it also change the type of Broadcast[Array[Byte]]] into Broadcast[Array[Array[Byte]]]].

Testing for support broadcast objects larger than 2G is slow and memory hungry, so this is tested manually, could be added into SparkPerf.

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

Closes #2659 from davies/huge and squashes the following commits:

7b57a14 [Davies Liu] add more tests for broadcast
28acff9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into huge
a2f6a02 [Davies Liu] bug fix
4820613 [Davies Liu] Merge branch 'master' of github.com:apache/spark into huge
5875c73 [Davies Liu] address comments
10a349b [Davies Liu] address comments
0c33016 [Davies Liu] Merge branch 'master' of github.com:apache/spark into huge
6182c8f [Davies Liu] Merge branch 'master' into huge
d94b68f [Davies Liu] Merge branch 'master' of github.com:apache/spark into huge
2514848 [Davies Liu] address comments
fda395b [Davies Liu] Merge branch 'master' of github.com:apache/spark into huge
1c2d928 [Davies Liu] fix scala style
091b107 [Davies Liu] broadcast objects larger than 2G
2014-11-18 16:17:51 -08:00
Kay Ousterhout 010bc86e40 [SPARK-4463] Add (de)select all button for add'l metrics.
This commit removes the behavior where when a user clicks
"Show additional metrics" on the stage page, all of the additional
metrics are automatically selected; now, collapsing and expanding
the additional metrics has no effect on which options are selected.
Instead, there's a "(De)select All" box at the top; checking this box
checks all additional metrics (and similarly, unchecking it unchecks
all additional metrics).

This commit is intended to be backported to 1.2, so that the additional
metrics behavior is not confusing to users.

Now when a user clicks the "Show additional metrics" menu, this is what
it looks like:
![image](https://cloud.githubusercontent.com/assets/1108612/5094347/1541ead6-6f15-11e4-8e8c-25a65ddbdfb2.png)

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #3331 from kayousterhout/SPARK-4463 and squashes the following commits:

9e17cea [Kay Ousterhout] Added italics
b731230 [Kay Ousterhout] [SPARK-4463] Add (de)select all button for add'l metrics.
2014-11-18 15:01:06 -08:00
Davies Liu e34f38ff1a [SPARK-4017] show progress bar in console
The progress bar will look like this:

![1___spark_job__85_250_finished__4_are_running___java_](https://cloud.githubusercontent.com/assets/40902/4854813/a02f44ac-6099-11e4-9060-7c73a73151d6.png)

In the right corner, the numbers are: finished tasks, running tasks, total tasks.

After the stage has finished, it will disappear.

The progress bar is only showed if logging level is WARN or higher (but progress in title is still showed), it can be turned off by spark.driver.showConsoleProgress.

Author: Davies Liu <davies@databricks.com>

Closes #3029 from davies/progress and squashes the following commits:

95336d5 [Davies Liu] Merge branch 'master' of github.com:apache/spark into progress
fc49ac8 [Davies Liu] address commentse
2e90f75 [Davies Liu] show multiple stages in same time
0081bcc [Davies Liu] address comments
38c42f1 [Davies Liu] fix tests
ab87958 [Davies Liu] disable progress bar during tests
30ac852 [Davies Liu] re-implement progress bar
b3f34e5 [Davies Liu] Merge branch 'master' of github.com:apache/spark into progress
6fd30ff [Davies Liu] show progress bar if no task finished in 500ms
e4e7344 [Davies Liu] refactor
e1f524d [Davies Liu] revert unnecessary change
a60477c [Davies Liu] Merge branch 'master' of github.com:apache/spark into progress
5cae3f2 [Davies Liu] fix style
ea49fe0 [Davies Liu] address comments
bc53d99 [Davies Liu] refactor
e6bb189 [Davies Liu] fix logging in sparkshell
7e7d4e7 [Davies Liu] address commments
5df26bb [Davies Liu] fix style
9e42208 [Davies Liu] show progress bar in console and title
2014-11-18 13:37:21 -08:00
Davies Liu 80f3177882 [SPARK-4404] remove sys.exit() in shutdown hook
If SparkSubmit die first, then bootstrapper will be blocked by shutdown hook. sys.exit() in a shutdown hook will cause some kind of dead lock.

cc andrewor14

Author: Davies Liu <davies@databricks.com>

Closes #3289 from davies/fix_bootstraper and squashes the following commits:

ea5cdd1 [Davies Liu] Merge branch 'master' of github.com:apache/spark into fix_bootstraper
e04b690 [Davies Liu] remove sys.exit in hook
4d11366 [Davies Liu] remove shutdown hook if subprocess die fist
2014-11-18 13:11:38 -08:00
Kousuke Saruta bfebfd8b28 [SPARK-4075][SPARK-4434] Fix the URI validation logic for Application Jar name.
This PR adds a regression test for SPARK-4434.

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

Closes #3326 from sarutak/add-triple-slash-testcase and squashes the following commits:

82bc9cc [Kousuke Saruta] Fixed wrong grammar in comment
9149027 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into add-triple-slash-testcase
c1c80ca [Kousuke Saruta] Fixed style
4f30210 [Kousuke Saruta] Modified comments
9e09da2 [Kousuke Saruta] Fixed URI validation for jar file
d4b99ef [Kousuke Saruta] [SPARK-4075] [Deploy] Jar url validation is not enough for Jar file
ac79906 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into add-triple-slash-testcase
6d4f47e [Kousuke Saruta] Added a test case as a regression check for SPARK-4434
2014-11-18 12:17:33 -08:00
Patrick Wendell c6e0c2ab1c SPARK-4466: Provide support for publishing Scala 2.11 artifacts to Maven
The maven release plug-in does not have support for publishing two separate sets of artifacts for a single release. Because of the way that Scala 2.11 support in Spark works, we have to write some customized code to do this. The good news is that the Maven release API is just a thin wrapper on doing git commits and pushing artifacts to the HTTP API of Apache's Sonatype server and this might overall make our deployment easier to understand.

This was already used for the 1.2 snapshot, so I think it is working well. One other nice thing is this could be pretty easily extended to publish nightly snapshots.

Author: Patrick Wendell <pwendell@gmail.com>

Closes #3332 from pwendell/releases and squashes the following commits:

2fedaed [Patrick Wendell] Automate the opening and closing of Sonatype repos
e2a24bb [Patrick Wendell] Fixing issue where we overrode non-spark version numbers
9df3a50 [Patrick Wendell] Adding TODO
1cc1749 [Patrick Wendell] Don't build the thriftserver for 2.11
933201a [Patrick Wendell] Make tagging of release commit eager
d0388a6 [Patrick Wendell] Support Scala 2.11 build
4f4dc62 [Patrick Wendell] Change to 2.11 should not be included when committing new patch
bf742e1 [Patrick Wendell] Minor fixes
ffa1df2 [Patrick Wendell] Adding a Scala 2.11 package to test it
9ac4381 [Patrick Wendell] Addressing TODO
b3105ff [Patrick Wendell] Removing commented out code
d906803 [Patrick Wendell] Small fix
3f4d985 [Patrick Wendell] More work
fcd54c2 [Patrick Wendell] Consolidating use of keys
df2af30 [Patrick Wendell] Changes to release stuff
2014-11-17 21:07:50 -08:00
Josh Rosen 0f3ceb56c7 [SPARK-4180] [Core] Prevent creation of multiple active SparkContexts
This patch adds error-detection logic to throw an exception when attempting to create multiple active SparkContexts in the same JVM, since this is currently unsupported and has been known to cause confusing behavior (see SPARK-2243 for more details).

**The solution implemented here is only a partial fix.**  A complete fix would have the following properties:

1. Only one SparkContext may ever be under construction at any given time.
2. Once a SparkContext has been successfully constructed, any subsequent construction attempts should fail until the active SparkContext is stopped.
3. If the SparkContext constructor throws an exception, then all resources created in the constructor should be cleaned up (SPARK-4194).
4. If a user attempts to create a SparkContext but the creation fails, then the user should be able to create new SparkContexts.

This PR only provides 2) and 4); we should be able to provide all of these properties, but the correct fix will involve larger changes to SparkContext's construction / initialization, so we'll target it for a different Spark release.

### The correct solution:

I think that the correct way to do this would be to move the construction of SparkContext's dependencies into a static method in the SparkContext companion object.  Specifically, we could make the default SparkContext constructor `private` and change it to accept a `SparkContextDependencies` object that contains all of SparkContext's dependencies (e.g. DAGScheduler, ContextCleaner, etc.).  Secondary constructors could call a method on the SparkContext companion object to create the `SparkContextDependencies` and pass the result to the primary SparkContext constructor.  For example:

```scala
class SparkContext private (deps: SparkContextDependencies) {
  def this(conf: SparkConf) {
    this(SparkContext.getDeps(conf))
  }
}

object SparkContext(
  private[spark] def getDeps(conf: SparkConf): SparkContextDependencies = synchronized {
    if (anotherSparkContextIsActive) { throw Exception(...) }
    var dagScheduler: DAGScheduler = null
    try {
        dagScheduler = new DAGScheduler(...)
        [...]
    } catch {
      case e: Exception =>
         Option(dagScheduler).foreach(_.stop())
          [...]
    }
    SparkContextDependencies(dagScheduler, ....)
  }
}
```

This gives us mutual exclusion and ensures that any resources created during the failed SparkContext initialization are properly cleaned up.

This indirection is necessary to maintain binary compatibility.  In retrospect, it would have been nice if SparkContext had no private constructors and could only be created through builder / factory methods on its companion object, since this buys us lots of flexibility and makes dependency injection easier.

### Alternative solutions:

As an alternative solution, we could refactor SparkContext's primary constructor to perform all object creation in a giant `try-finally` block.  Unfortunately, this will require us to turn a bunch of `vals` into `vars` so that they can be assigned from the `try` block.  If we still want `vals`, we could wrap each `val` in its own `try` block (since the try block can return a value), but this will lead to extremely messy code and won't guard against the introduction of future code which doesn't properly handle failures.

The more complex approach outlined above gives us some nice dependency injection benefits, so I think that might be preferable to a `var`-ification.

### This PR's solution:

- At the start of the constructor, check whether some other SparkContext is active; if so, throw an exception.
- If another SparkContext might be under construction (or has thrown an exception during construction), allow the new SparkContext to begin construction but log a warning (since resources might have been leaked from a failed creation attempt).
- At the end of the SparkContext constructor, check whether some other SparkContext constructor has raced and successfully created an active context.  If so, throw an exception.

This guarantees that no two SparkContexts will ever be active and exposed to users (since we check at the very end of the constructor).  If two threads race to construct SparkContexts, then one of them will win and another will throw an exception.

This exception can be turned into a warning by setting `spark.driver.allowMultipleContexts = true`.  The exception is disabled in unit tests, since there are some suites (such as Hive) that may require more significant refactoring to clean up their SparkContexts.  I've made a few changes to other suites' test fixtures to properly clean up SparkContexts so that the unit test logs contain fewer warnings.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #3121 from JoshRosen/SPARK-4180 and squashes the following commits:

23c7123 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4180
d38251b [Josh Rosen] Address latest round of feedback.
c0987d3 [Josh Rosen] Accept boolean instead of SparkConf in methods.
85a424a [Josh Rosen] Incorporate more review feedback.
372d0d3 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4180
f5bb78c [Josh Rosen] Update mvn build, too.
d809cb4 [Josh Rosen] Improve handling of failed SparkContext creation attempts.
79a7e6f [Josh Rosen] Fix commented out test
a1cba65 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4180
7ba6db8 [Josh Rosen] Add utility to set system properties in tests.
4629d5c [Josh Rosen] Set spark.driver.allowMultipleContexts=true in tests.
ed17e14 [Josh Rosen] Address review feedback; expose hack workaround for existing unit tests.
1c66070 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4180
06c5c54 [Josh Rosen] Add / improve SparkContext cleanup in streaming BasicOperationsSuite
d0437eb [Josh Rosen] StreamingContext.stop() should stop SparkContext even if StreamingContext has not been started yet.
c4d35a2 [Josh Rosen] Log long form of creation site to aid debugging.
918e878 [Josh Rosen] Document "one SparkContext per JVM" limitation.
afaa7e3 [Josh Rosen] [SPARK-4180] Prevent creations of multiple active SparkContexts.
2014-11-17 12:48:18 -08:00
Andrew Or dbb9da5c3d Revert "[SPARK-4075] [Deploy] Jar url validation is not enough for Jar file"
This reverts commit 098f83c7cc.
2014-11-17 11:24:28 -08:00
Prashant Sharma 5c92d47ad2 SPARK-4445, Don't display storage level in toDebugString unless RDD is persisted.
Author: Prashant Sharma <prashant.s@imaginea.com>

Closes #3310 from ScrapCodes/SPARK-4445/rddDebugStringFix and squashes the following commits:

4e57c52 [Prashant Sharma] SPARK-4445, Don't display storage level in toDebugString unless RDD is persisted
2014-11-17 10:40:33 -08:00
Josh Rosen 7850e0c707 [SPARK-4393] Fix memory leak in ConnectionManager ACK timeout TimerTasks; use HashedWheelTimer
This patch is intended to fix a subtle memory leak in ConnectionManager's ACK timeout TimerTasks: in the old code, each TimerTask held a reference to the message being sent and a cancelled TimerTask won't necessarily be garbage-collected until it's scheduled to run, so this caused huge buildups of messages that weren't garbage collected until their timeouts expired, leading to OOMs.

This patch addresses this problem by capturing only the message ID in the TimerTask instead of the whole message, and by keeping a WeakReference to the promise in the TimerTask.  I've also modified this code to use Netty's HashedWheelTimer, whose performance characteristics should be better for this use-case.

Thanks to cristianopris for narrowing down this issue!

Author: Josh Rosen <joshrosen@databricks.com>

Closes #3259 from JoshRosen/connection-manager-timeout-bugfix and squashes the following commits:

afcc8d6 [Josh Rosen] Address rxin's review feedback.
2a2e92d [Josh Rosen] Keep only WeakReference to promise in TimerTask;
0f0913b [Josh Rosen] Spelling fix: timout => timeout
3200c33 [Josh Rosen] Use Netty HashedWheelTimer
f847dd4 [Josh Rosen] Don't capture entire message in ACK timeout task.
2014-11-16 00:44:15 -08:00
Josh Rosen 40eb8b6ef3 [SPARK-2321] Several progress API improvements / refactorings
This PR refactors / extends the status API introduced in #2696.

- Change StatusAPI from a mixin trait to a class.  Before, the new status API methods were directly accessible through SparkContext, whereas now they're accessed through a `sc.statusAPI` field.  As long as we were going to add these methods directly to SparkContext, the mixin trait seemed like a good idea, but this might be simpler to reason about and may avoid pitfalls that I've run into while attempting to refactor other parts of SparkContext to use mixins (see #3071, for example).
- Change the name from SparkStatusAPI to SparkStatusTracker.
- Make `getJobIdsForGroup(null)` return ids for jobs that aren't associated with any job group.
- Add `getActiveStageIds()` and `getActiveJobIds()` methods that return the ids of whatever's currently active in this SparkContext.  This should simplify davies's progress bar code.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #3197 from JoshRosen/progress-api-improvements and squashes the following commits:

30b0afa [Josh Rosen] Rename SparkStatusAPI to SparkStatusTracker.
d1b08d8 [Josh Rosen] Add missing newlines
2cc7353 [Josh Rosen] Add missing file.
d5eab1f [Josh Rosen] Add getActive[Stage|Job]Ids() methods.
a227984 [Josh Rosen] getJobIdsForGroup(null) should return jobs for default group
c47e294 [Josh Rosen] Remove StatusAPI mixin trait.
2014-11-14 23:46:25 -08:00
Kousuke Saruta 60969b0336 [SPARK-4260] Httpbroadcast should set connection timeout.
Httpbroadcast sets read timeout but doesn't set connection timeout.

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

Closes #3122 from sarutak/httpbroadcast-timeout and squashes the following commits:

c7f3a56 [Kousuke Saruta] Added Connection timeout for Http Connection to HttpBroadcast.scala
2014-11-14 22:36:56 -08:00
zsxwing 861223ee5b [SPARK-4363][Doc] Update the Broadcast example
Author: zsxwing <zsxwing@gmail.com>

Closes #3226 from zsxwing/SPARK-4363 and squashes the following commits:

8109914 [zsxwing] Update the Broadcast example
2014-11-14 22:28:48 -08:00
zsxwing dba1405823 [SPARK-4379][Core] Change Exception to SparkException in checkpoint
It's better to change to SparkException. However, it's a breaking change since it will change the exception type.

Author: zsxwing <zsxwing@gmail.com>

Closes #3241 from zsxwing/SPARK-4379 and squashes the following commits:

409f3af [zsxwing] Change Exception to SparkException in checkpoint
2014-11-14 22:25:41 -08:00
Davies Liu 7fe08b43c7 [SPARK-4415] [PySpark] JVM should exit after Python exit
When JVM is started in a Python process, it should exit once the stdin is closed.

test: add spark.driver.memory in conf/spark-defaults.conf

```
daviesdm:~/work/spark$ cat conf/spark-defaults.conf
spark.driver.memory       8g
daviesdm:~/work/spark$ bin/pyspark
>>> quit
daviesdm:~/work/spark$ jps
4931 Jps
286
daviesdm:~/work/spark$ python wc.py
943738
0.719928026199
daviesdm:~/work/spark$ jps
286
4990 Jps
```

Author: Davies Liu <davies@databricks.com>

Closes #3274 from davies/exit and squashes the following commits:

df0e524 [Davies Liu] address comments
ce8599c [Davies Liu] address comments
050651f [Davies Liu] JVM should exit after Python exit
2014-11-14 20:14:33 -08:00
WangTao 303a4e4d23 [SPARK-4404]SparkSubmitDriverBootstrapper should stop after its SparkSubmit sub-proc...
...ess ends

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

When we have spark.driver.extra* or spark.driver.memory in SPARK_SUBMIT_PROPERTIES_FILE, spark-class will use SparkSubmitDriverBootstrapper to launch driver.
If we get process id of SparkSubmitDriverBootstrapper and wanna kill it during its running, we expect its SparkSubmit sub-process stop also.

Author: WangTao <barneystinson@aliyun.com>
Author: WangTaoTheTonic <barneystinson@aliyun.com>

Closes #3266 from WangTaoTheTonic/killsubmit and squashes the following commits:

e03eba5 [WangTaoTheTonic] add comments
57b5ca1 [WangTao] SparkSubmitDriverBootstrapper should stop after its SparkSubmit sub-process ends
2014-11-14 20:11:51 -08:00
Sandy Ryza ad42b28324 SPARK-4214. With dynamic allocation, avoid outstanding requests for more...
... executors than pending tasks need.

WIP. Still need to add and fix tests.

Author: Sandy Ryza <sandy@cloudera.com>

Closes #3204 from sryza/sandy-spark-4214 and squashes the following commits:

35cf0e0 [Sandy Ryza] Add comment
13b53df [Sandy Ryza] Review feedback
067465f [Sandy Ryza] Whitespace fix
6ae080c [Sandy Ryza] Add tests and get num pending tasks from ExecutorAllocationListener
531e2b6 [Sandy Ryza] SPARK-4214. With dynamic allocation, avoid outstanding requests for more executors than pending tasks need.
2014-11-14 15:51:40 -08:00
Jeff Hammerbacher c258db9ed4 Update failed assert text to match code in SizeEstimatorSuite
Author: Jeff Hammerbacher <jeff.hammerbacher@gmail.com>

Closes #3242 from hammer/patch-1 and squashes the following commits:

f88d635 [Jeff Hammerbacher] Update failed assert text to match code in SizeEstimatorSuite
2014-11-14 13:37:48 -08:00
zsxwing 156cf3333d [SPARK-4313][WebUI][Yarn] Fix link issue of the executor thread dump page in yarn-cluster mode
In yarn-cluster mode, the Web UI is running behind a yarn proxy server. Some features(or bugs?) of yarn proxy server will break the links for thread dump.

1. Yarn proxy server will do http redirect internally, so if opening `http://example.com:8088/cluster/app/application_1415344371838_0012/executors`, it will fetch `http://example.com:8088/cluster/app/application_1415344371838_0012/executors/` and return the content but won't change the link in the browser. Then when a user clicks `Thread Dump`, it will jump to `http://example.com:8088/proxy/application_1415344371838_0012/threadDump/?executorId=2`. This is a wrong link. The correct link should be `http://example.com:8088/proxy/application_1415344371838_0012/executors/threadDump/?executorId=2`.

Adding "/" to the tab links will fix it.

2. Yarn proxy server has a bug about the URL encode/decode. When a user accesses `http://example.com:8088/proxy/application_1415344371838_0006/executors/threadDump/?executorId=%3Cdriver%3E`, the yarn proxy server will require `http://example.com:36429/executors/threadDump/?executorId=%25253Cdriver%25253E`. But Spark web server expects `http://example.com:36429/executors/threadDump/?executorId=%3Cdriver%3E`. Related to [YARN-2844](https://issues.apache.org/jira/browse/YARN-2844).

For now, it's a tricky approach to bypass the yarn bug.

![threaddump](https://cloud.githubusercontent.com/assets/1000778/4972567/d1ccba64-68ad-11e4-983e-257530cef35a.png)

Author: zsxwing <zsxwing@gmail.com>

Closes #3183 from zsxwing/SPARK-4313 and squashes the following commits:

3379ca8 [zsxwing] Encode the executor id in the thread dump link and update the comment
abfa063 [zsxwing] Fix link issue of the executor thread dump page in yarn-cluster mode
2014-11-14 13:36:13 -08:00
Hong Shen 0c56a039a9 [Spark Core] SPARK-4380 Edit spilling log from MB to B
https://issues.apache.org/jira/browse/SPARK-4380

Author: Hong Shen <hongshen@tencent.com>

Closes #3243 from shenh062326/spark_change and squashes the following commits:

4653378 [Hong Shen] Edit spilling log from MB to B
21ee960 [Hong Shen] Edit spilling log from MB to B
e9145e8 [Hong Shen] Edit spilling log from MB to B
da761c2 [Hong Shen] Edit spilling log from MB to B
946351c [Hong Shen] Edit spilling log from MB to B
2014-11-14 13:29:41 -08:00
zsxwing 825709a0b8 [SPARK-4310][WebUI] Sort 'Submitted' column in Stage page by time
Author: zsxwing <zsxwing@gmail.com>

Closes #3179 from zsxwing/SPARK-4310 and squashes the following commits:

b0d29f5 [zsxwing] Sort 'Submitted' column in Stage page by time
2014-11-13 14:37:04 -08:00
Aaron Davidson b9e1c2eb9b [SPARK-4370] [Core] Limit number of Netty cores based on executor size
Author: Aaron Davidson <aaron@databricks.com>

Closes #3155 from aarondav/conf and squashes the following commits:

7045e77 [Aaron Davidson] Add mesos comment
4770f6e [Aaron Davidson] [SPARK-4370] [Core] Limit number of Netty cores based on executor size
2014-11-12 18:46:37 -08:00
Davies Liu d7d54a44e3 [SPARK-2672] support compressed file in wholeTextFile
The wholeFile() can not read compressed files, it should be, just like textFile().

Author: Davies Liu <davies@databricks.com>

Closes #3005 from davies/whole and squashes the following commits:

a43fcfb [Davies Liu] remove semicolon
c83571a [Davies Liu] remove = if return type is Unit
83c844f [Davies Liu] Merge branch 'master' of github.com:apache/spark into whole
22e8b3e [Davies Liu] support compressed file in wholeTextFile
2014-11-12 15:58:12 -08:00
Andrew Or 6e3c5a296c [Test] Better exception message from SparkSubmitSuite
Before:
```
Exception in thread "main" java.lang.Exception: Could not load user defined classes inside of executors
	at org.apache.spark.deploy.JarCreationTest$.main(SparkSubmitSuite.scala:471)
	at org.apache.spark.deploy.JarCreationTest.main(SparkSubmitSuite.scala)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
```
After:
```
Exception in thread "main" java.lang.Exception: Could not load user class from jar:
java.lang.UnsupportedClassVersionError: SparkSubmitClassA : Unsupported major.minor version 51.0
	java.lang.ClassLoader.defineClass1(Native Method)
	java.lang.ClassLoader.defineClass(ClassLoader.java:643)
	...
	at org.apache.spark.deploy.JarCreationTest$.main(SparkSubmitSuite.scala:472)
	at org.apache.spark.deploy.JarCreationTest.main(SparkSubmitSuite.scala)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
```

Author: Andrew Or <andrew@databricks.com>

Closes #3212 from andrewor14/submit-suite-message and squashes the following commits:

7779248 [Andrew Or] Format exception
8fe6719 [Andrew Or] Better exception message from failed test
2014-11-12 13:35:48 -08:00
Prashant Sharma daaca14c16 Support cross building for Scala 2.11
Let's give this another go using a version of Hive that shades its JLine dependency.

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

Closes #3159 from pwendell/scala-2.11-prashant and squashes the following commits:

e93aa3e [Patrick Wendell] Restoring -Phive-thriftserver profile and cleaning up build script.
f65d17d [Patrick Wendell] Fixing build issue due to merge conflict
a8c41eb [Patrick Wendell] Reverting dev/run-tests back to master state.
7a6eb18 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into scala-2.11-prashant
583aa07 [Prashant Sharma] REVERT ME: removed hive thirftserver
3680e58 [Prashant Sharma] Revert "REVERT ME: Temporarily removing some Cli tests."
935fb47 [Prashant Sharma] Revert "Fixed by disabling a few tests temporarily."
925e90f [Prashant Sharma] Fixed by disabling a few tests temporarily.
2fffed3 [Prashant Sharma] Exclude groovy from sbt build, and also provide a way for such instances in future.
8bd4e40 [Prashant Sharma] Switched to gmaven plus, it fixes random failures observer with its predecessor gmaven.
5272ce5 [Prashant Sharma] SPARK_SCALA_VERSION related bugs.
2121071 [Patrick Wendell] Migrating version detection to PySpark
b1ed44d [Patrick Wendell] REVERT ME: Temporarily removing some Cli tests.
1743a73 [Patrick Wendell] Removing decimal test that doesn't work with Scala 2.11
f5cad4e [Patrick Wendell] Add Scala 2.11 docs
210d7e1 [Patrick Wendell] Revert "Testing new Hive version with shaded jline"
48518ce [Patrick Wendell] Remove association of Hive and Thriftserver profiles.
e9d0a06 [Patrick Wendell] Revert "Enable thritfserver for Scala 2.10 only"
67ec364 [Patrick Wendell] Guard building of thriftserver around Scala 2.10 check
8502c23 [Patrick Wendell] Enable thritfserver for Scala 2.10 only
e22b104 [Patrick Wendell] Small fix in pom file
ec402ab [Patrick Wendell] Various fixes
0be5a9d [Patrick Wendell] Testing new Hive version with shaded jline
4eaec65 [Prashant Sharma] Changed scripts to ignore target.
5167bea [Prashant Sharma] small correction
a4fcac6 [Prashant Sharma] Run against scala 2.11 on jenkins.
80285f4 [Prashant Sharma] MAven equivalent of setting spark.executor.extraClasspath during tests.
034b369 [Prashant Sharma] Setting test jars on executor classpath during tests from sbt.
d4874cb [Prashant Sharma] Fixed Python Runner suite. null check should be first case in scala 2.11.
6f50f13 [Prashant Sharma] Fixed build after rebasing with master. We should use ${scala.binary.version} instead of just 2.10
e56ca9d [Prashant Sharma] Print an error if build for 2.10 and 2.11 is spotted.
937c0b8 [Prashant Sharma] SCALA_VERSION -> SPARK_SCALA_VERSION
cb059b0 [Prashant Sharma] Code review
0476e5e [Prashant Sharma] Scala 2.11 support with repl and all build changes.
2014-11-11 21:36:48 -08:00
Timothy Chen a878660d2d SPARK-2269 Refactor mesos scheduler resourceOffers and add unit test
Author: Timothy Chen <tnachen@gmail.com>

Closes #1487 from tnachen/resource_offer_refactor and squashes the following commits:

4ea5dec [Timothy Chen] Rebase from master and address comments
9ccab09 [Timothy Chen] Address review comments
e6494dc [Timothy Chen] Refactor class loading
8207428 [Timothy Chen] Refactor mesos scheduler resourceOffers and add unit test
2014-11-11 14:29:18 -08:00
Prashant Sharma deefd9d737 SPARK-1830 Deploy failover, Make Persistence engine and LeaderAgent Pluggable
Author: Prashant Sharma <prashant.s@imaginea.com>

Closes #771 from ScrapCodes/deploy-failover-pluggable and squashes the following commits:

29ba440 [Prashant Sharma] fixed a compilation error
fef35ec [Prashant Sharma] Code review
57ee6f0 [Prashant Sharma] SPARK-1830 Deploy failover, Make Persistence engine and LeaderAgent Pluggable.
2014-11-11 09:29:48 -08:00
Reynold Xin ef29a9a9aa [SPARK-4307] Initialize FileDescriptor lazily in FileRegion.
Netty's DefaultFileRegion requires a FileDescriptor in its constructor, which means we need to have a opened file handle. In super large workloads, this could lead to too many open files due to the way these file descriptors are cleaned. This pull request creates a new LazyFileRegion that initializes the FileDescriptor when we are sending data for the first time.

Author: Reynold Xin <rxin@databricks.com>
Author: Reynold Xin <rxin@apache.org>

Closes #3172 from rxin/lazyFD and squashes the following commits:

0bdcdc6 [Reynold Xin] Added reference to Netty's DefaultFileRegion
d4564ae [Reynold Xin] Added SparkConf to the ctor argument of IndexShuffleBlockManager.
6ed369e [Reynold Xin] Code review feedback.
04cddc8 [Reynold Xin] [SPARK-4307] Initialize FileDescriptor lazily in FileRegion.
2014-11-11 00:25:31 -08:00
Patrick Wendell 6e7a309b81 Revert "[SPARK-2703][Core]Make Tachyon related unit tests execute without deploying a Tachyon system locally."
This reverts commit bd86cb1738.
2014-11-10 14:56:06 -08:00
Niklas Wilcke ed8bf1eac5 [SPARK-4169] [Core] Accommodate non-English Locales in unit tests
For me the core tests failed because there are two locale dependent parts in the code.
Look at the Jira ticket for details.

Why is it necessary to check the exception message in isBindCollision in
https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/util/Utils.scala#L1686
?

Author: Niklas Wilcke <1wilcke@informatik.uni-hamburg.de>

Closes #3036 from numbnut/core-test-fix and squashes the following commits:

1fb0d04 [Niklas Wilcke] Fixing locale dependend code and tests
2014-11-10 11:37:38 -08:00
RongGu bd86cb1738 [SPARK-2703][Core]Make Tachyon related unit tests execute without deploying a Tachyon system locally.
Make Tachyon related unit tests execute without deploying a Tachyon system locally.

Author: RongGu <gurongwalker@gmail.com>

Closes #3030 from RongGu/SPARK-2703 and squashes the following commits:

ad08827 [RongGu] Make Tachyon related unit tests execute without deploying a Tachyon system locally
2014-11-09 23:48:15 -08:00
Sandy Ryza 3c2cff4b94 SPARK-3179. Add task OutputMetrics.
Author: Sandy Ryza <sandy@cloudera.com>

This patch had conflicts when merged, resolved by
Committer: Kay Ousterhout <kayousterhout@gmail.com>

Closes #2968 from sryza/sandy-spark-3179 and squashes the following commits:

dce4784 [Sandy Ryza] More review feedback
8d350d1 [Sandy Ryza] Fix test against Hadoop 2.5+
e7c74d0 [Sandy Ryza] More review feedback
6cff9c4 [Sandy Ryza] Review feedback
fb2dde0 [Sandy Ryza] SPARK-3179
2014-11-09 22:29:03 -08:00
Sean Owen f8e5732307 SPARK-1209 [CORE] (Take 2) SparkHadoop{MapRed,MapReduce}Util should not use package org.apache.hadoop
andrewor14 Another try at SPARK-1209, to address https://github.com/apache/spark/pull/2814#issuecomment-61197619

I successfully tested with `mvn -Dhadoop.version=1.0.4 -DskipTests clean package; mvn -Dhadoop.version=1.0.4 test` I assume that is what failed Jenkins last time. I also tried `-Dhadoop.version1.2.1` and `-Phadoop-2.4 -Pyarn -Phive` for more coverage.

So this is why the class was put in `org.apache.hadoop` to begin with, I assume. One option is to leave this as-is for now and move it only when Hadoop 1.0.x support goes away.

This is the other option, which adds a call to force the constructor to be public at run-time. It's probably less surprising than putting Spark code in `org.apache.hadoop`, but, does involve reflection. A `SecurityManager` might forbid this, but it would forbid a lot of stuff Spark does. This would also only affect Hadoop 1.0.x it seems.

Author: Sean Owen <sowen@cloudera.com>

Closes #3048 from srowen/SPARK-1209 and squashes the following commits:

0d48f4b [Sean Owen] For Hadoop 1.0.x, make certain constructors public, which were public in later versions
466e179 [Sean Owen] Disable MIMA warnings resulting from moving the class -- this was also part of the PairRDDFunctions type hierarchy though?
eb61820 [Sean Owen] Move SparkHadoopMapRedUtil / SparkHadoopMapReduceUtil from org.apache.hadoop to org.apache.spark
2014-11-09 22:11:20 -08:00
Sean Owen d1362659ef SPARK-1344 [DOCS] Scala API docs for top methods
Use "k" in javadoc of top and takeOrdered to avoid confusion with type K in pair RDDs. I think this resolves the discussion in SPARK-1344.

Author: Sean Owen <sowen@cloudera.com>

Closes #3168 from srowen/SPARK-1344 and squashes the following commits:

6963fcc [Sean Owen] Use "k" in javadoc of top and takeOrdered to avoid confusion with type K in pair RDDs
2014-11-09 17:42:08 -08:00
Cheng Lian 86e9eaa3f0 [SPARK-4225][SQL] Resorts to SparkContext.version to inspect Spark version
This PR resorts to `SparkContext.version` rather than META-INF/MANIFEST.MF in the assembly jar to inspect Spark version. Currently, when built with Maven, the MANIFEST.MF file in the assembly jar is incorrectly replaced by Guava 15.0 MANIFEST.MF, probably because of the assembly/shading tricks.

Another related PR is #3103, which tries to fix the MANIFEST issue.

Author: Cheng Lian <lian@databricks.com>

Closes #3105 from liancheng/spark-4225 and squashes the following commits:

d9585e1 [Cheng Lian] Resorts to SparkContext.version to inspect Spark version
2014-11-07 11:45:25 -08:00
Aaron Davidson d4fa04e50d [SPARK-4187] [Core] Switch to binary protocol for external shuffle service messages
This PR elimiantes the network package's usage of the Java serializer and replaces it with Encodable, which is a lightweight binary protocol. Each message is preceded by a type id, which will allow us to change messages (by only adding new ones), or to change the format entirely by switching to a special id (such as -1).

This protocol has the advantage over Java that we can guarantee that messages will remain compatible across compiled versions and JVMs, though it does not provide a clean way to do schema migration. In the future, it may be good to use a more heavy-weight serialization format like protobuf, thrift, or avro, but these all add several dependencies which are unnecessary at the present time.

Additionally this unifies the RPC messages of NettyBlockTransferService and ExternalShuffleClient.

Author: Aaron Davidson <aaron@databricks.com>

Closes #3146 from aarondav/free and squashes the following commits:

ed1102a [Aaron Davidson] Remove some unused imports
b8e2a49 [Aaron Davidson] Add appId to test
538f2a3 [Aaron Davidson] [SPARK-4187] [Core] Switch to binary protocol for external shuffle service messages
2014-11-07 09:42:21 -08:00
zsxwing 3abdb1b24a [SPARK-4204][Core][WebUI] Change Utils.exceptionString to contain the inner exceptions and make the error information in Web UI more friendly
This PR fixed `Utils.exceptionString` to output the full exception information. However, the stack trace may become very huge, so I also updated the Web UI to collapse the error information by default (display the first line and clicking `+detail` will display the full info).

Here are the screenshots:

Stages:
![stages](https://cloud.githubusercontent.com/assets/1000778/4882441/66d8cc68-6356-11e4-8346-6318677d9470.png)

Details for one stage:
![stage](https://cloud.githubusercontent.com/assets/1000778/4882513/1311043c-6357-11e4-8804-ca14240a9145.png)

The full information in the gray text field is:
```Java
org.apache.spark.shuffle.FetchFailedException: Connection reset by peer
	at org.apache.spark.shuffle.hash.BlockStoreShuffleFetcher$.org$apache$spark$shuffle$hash$BlockStoreShuffleFetcher$$unpackBlock$1(BlockStoreShuffleFetcher.scala:67)
	at org.apache.spark.shuffle.hash.BlockStoreShuffleFetcher$$anonfun$3.apply(BlockStoreShuffleFetcher.scala:83)
	at org.apache.spark.shuffle.hash.BlockStoreShuffleFetcher$$anonfun$3.apply(BlockStoreShuffleFetcher.scala:83)
	at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371)
	at org.apache.spark.util.CompletionIterator.hasNext(CompletionIterator.scala:30)
	at org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:39)
	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:327)
	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:327)
	at org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:129)
	at org.apache.spark.rdd.CoGroupedRDD$$anonfun$compute$5.apply(CoGroupedRDD.scala:160)
	at org.apache.spark.rdd.CoGroupedRDD$$anonfun$compute$5.apply(CoGroupedRDD.scala:159)
	at scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:772)
	at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
	at scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:771)
	at org.apache.spark.rdd.CoGroupedRDD.compute(CoGroupedRDD.scala:159)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:263)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:230)
	at org.apache.spark.rdd.MappedValuesRDD.compute(MappedValuesRDD.scala:31)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:263)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:230)
	at org.apache.spark.rdd.FlatMappedValuesRDD.compute(FlatMappedValuesRDD.scala:31)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:263)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:230)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)
	at org.apache.spark.scheduler.Task.run(Task.scala:56)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:189)
	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
	at java.lang.Thread.run(Thread.java:662)
Caused by: java.io.IOException: Connection reset by peer
	at sun.nio.ch.FileDispatcher.read0(Native Method)
	at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:21)
	at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:198)
	at sun.nio.ch.IOUtil.read(IOUtil.java:166)
	at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:245)
	at io.netty.buffer.PooledUnsafeDirectByteBuf.setBytes(PooledUnsafeDirectByteBuf.java:311)
	at io.netty.buffer.AbstractByteBuf.writeBytes(AbstractByteBuf.java:881)
	at io.netty.channel.socket.nio.NioSocketChannel.doReadBytes(NioSocketChannel.java:225)
	at io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:119)
	at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
	at io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
	at io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
	at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
	at io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:116)
	... 1 more
```

/cc aarondav

Author: zsxwing <zsxwing@gmail.com>

Closes #3073 from zsxwing/SPARK-4204 and squashes the following commits:

176d1e3 [zsxwing] Add comments to explain the stack trace difference
ca509d3 [zsxwing] Add fullStackTrace to the constructor of ExceptionFailure
a07057b [zsxwing] Core style fix
dfb0032 [zsxwing] Backward compatibility for old history server
1e50f71 [zsxwing] Update as per review and increase the max height of the stack trace details
94f2566 [zsxwing] Change Utils.exceptionString to contain the inner exceptions and make the error information in Web UI more friendly
2014-11-06 21:52:38 -08:00
Aaron Davidson 48a19a6dba [SPARK-4236] Cleanup removed applications' files in shuffle service
This relies on a hook from whoever is hosting the shuffle service to invoke removeApplication() when the application is completed. Once invoked, we will clean up all the executors' shuffle directories we know about.

Author: Aaron Davidson <aaron@databricks.com>

Closes #3126 from aarondav/cleanup and squashes the following commits:

33a64a9 [Aaron Davidson] Missing brace
e6e428f [Aaron Davidson] Address comments
16a0d27 [Aaron Davidson] Cleanup
e4df3e7 [Aaron Davidson] [SPARK-4236] Cleanup removed applications' files in shuffle service
2014-11-06 19:54:32 -08:00
Aaron Davidson f165b2bbf5 [SPARK-4188] [Core] Perform network-level retry of shuffle file fetches
This adds a RetryingBlockFetcher to the NettyBlockTransferService which is wrapped around our typical OneForOneBlockFetcher, adding retry logic in the event of an IOException.

This sort of retry allows us to avoid marking an entire executor as failed due to garbage collection or high network load.

TODO:
- [x] unit tests
- [x] put in ExternalShuffleClient too

Author: Aaron Davidson <aaron@databricks.com>

Closes #3101 from aarondav/retry and squashes the following commits:

72a2a32 [Aaron Davidson] Add that we should remove the condition around the retry thingy
c7fd107 [Aaron Davidson] Fix unit tests
e80e4c2 [Aaron Davidson] Address initial comments
6f594cd [Aaron Davidson] Fix unit test
05ff43c [Aaron Davidson] Add to external shuffle client and add unit test
66e5a24 [Aaron Davidson] [SPARK-4238] [Core] Perform network-level retry of shuffle file fetches
2014-11-06 18:39:14 -08:00
Aaron Davidson 6e9ef10fd7 [SPARK-4277] Support external shuffle service on Standalone Worker
Author: Aaron Davidson <aaron@databricks.com>

Closes #3142 from aarondav/worker and squashes the following commits:

3780bd7 [Aaron Davidson] Address comments
2dcdfc1 [Aaron Davidson] Add private[worker]
47f49d3 [Aaron Davidson] NettyBlockTransferService shouldn't care about app ids (it's only b/t executors)
258417c [Aaron Davidson] [SPARK-4277] Support external shuffle service on executor
2014-11-06 17:20:46 -08:00
Aaron Davidson 23eaf0e12f [SPARK-4264] Completion iterator should only invoke callback once
Author: Aaron Davidson <aaron@databricks.com>

Closes #3128 from aarondav/compiter and squashes the following commits:

698e4be [Aaron Davidson] [SPARK-4264] Completion iterator should only invoke callback once
2014-11-06 10:45:46 -08:00
Davies Liu b41a39e240 [SPARK-4186] add binaryFiles and binaryRecords in Python
add binaryFiles() and binaryRecords() in Python
```
binaryFiles(self, path, minPartitions=None):
    :: Developer API ::

    Read a directory of binary files from HDFS, a local file system
    (available on all nodes), or any Hadoop-supported file system URI
    as a byte array. Each file is read as a single record and returned
    in a key-value pair, where the key is the path of each file, the
    value is the content of each file.

    Note: Small files are preferred, large file is also allowable, but
    may cause bad performance.

binaryRecords(self, path, recordLength):
    Load data from a flat binary file, assuming each record is a set of numbers
    with the specified numerical format (see ByteBuffer), and the number of
    bytes per record is constant.

    :param path: Directory to the input data files
    :param recordLength: The length at which to split the records
```

Author: Davies Liu <davies@databricks.com>

Closes #3078 from davies/binary and squashes the following commits:

cd0bdbd [Davies Liu] Merge branch 'master' of github.com:apache/spark into binary
3aa349b [Davies Liu] add experimental notes
24e84b6 [Davies Liu] Merge branch 'master' of github.com:apache/spark into binary
5ceaa8a [Davies Liu] Merge branch 'master' of github.com:apache/spark into binary
1900085 [Davies Liu] bugfix
bb22442 [Davies Liu] add binaryFiles and binaryRecords in Python
2014-11-06 00:22:19 -08:00
Kay Ousterhout 5f27ae16d5 [SPARK-4255] Fix incorrect table striping
This commit stripes table rows after hiding some rows, to
ensure that rows are correct striped to alternate white
and grey even when rows are hidden by default.

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #3117 from kayousterhout/striping and squashes the following commits:

be6e10a [Kay Ousterhout] [SPARK-4255] Fix incorrect table striping
2014-11-06 00:03:03 -08:00
Brenden Matthews cb0eae3b78 [SPARK-4158] Fix for missing resources.
Mesos offers may not contain all resources, and Spark needs to check to
ensure they are present and sufficient.  Spark may throw an erroneous
exception when resources aren't present.

Author: Brenden Matthews <brenden@diddyinc.com>

Closes #3024 from brndnmtthws/fix-mesos-resource-misuse and squashes the following commits:

e5f9580 [Brenden Matthews] [SPARK-4158] Fix for missing resources.
2014-11-05 16:02:44 -08:00
Jongyoul Lee f7ac8c2b1d SPARK-3223 runAsSparkUser cannot change HDFS write permission properly i...
...n mesos cluster mode

- change master newer

Author: Jongyoul Lee <jongyoul@gmail.com>

Closes #3034 from jongyoul/SPARK-3223 and squashes the following commits:

42b2ed3 [Jongyoul Lee] SPARK-3223 runAsSparkUser cannot change HDFS write permission properly in mesos cluster mode - change master newer
2014-11-05 15:49:42 -08:00
Andrew Or 61a5cced04 [SPARK-3797] Run external shuffle service in Yarn NM
This creates a new module `network/yarn` that depends on `network/shuffle` recently created in #3001. This PR introduces a custom Yarn auxiliary service that runs the external shuffle service. As of the changes here this shuffle service is required for using dynamic allocation with Spark.

This is still WIP mainly because it doesn't handle security yet. I have tested this on a stable Yarn cluster.

Author: Andrew Or <andrew@databricks.com>

Closes #3082 from andrewor14/yarn-shuffle-service and squashes the following commits:

ef3ddae [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-shuffle-service
0ee67a2 [Andrew Or] Minor wording suggestions
1c66046 [Andrew Or] Remove unused provided dependencies
0eb6233 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-shuffle-service
6489db5 [Andrew Or] Try catch at the right places
7b71d8f [Andrew Or] Add detailed java docs + reword a few comments
d1124e4 [Andrew Or] Add security to shuffle service (INCOMPLETE)
5f8a96f [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-shuffle-service
9b6e058 [Andrew Or] Address various feedback
f48b20c [Andrew Or] Fix tests again
f39daa6 [Andrew Or] Do not make network-yarn an assembly module
761f58a [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-shuffle-service
15a5b37 [Andrew Or] Fix build for Hadoop 1.x
baff916 [Andrew Or] Fix tests
5bf9b7e [Andrew Or] Address a few minor comments
5b419b8 [Andrew Or] Add missing license header
804e7ff [Andrew Or] Include the Yarn shuffle service jar in the distribution
cd076a4 [Andrew Or] Require external shuffle service for dynamic allocation
ea764e0 [Andrew Or] Connect to Yarn shuffle service only if it's enabled
1bf5109 [Andrew Or] Use the shuffle service port specified through hadoop config
b4b1f0c [Andrew Or] 4 tabs -> 2 tabs
43dcb96 [Andrew Or] First cut integration of shuffle service with Yarn aux service
b54a0c4 [Andrew Or] Initial skeleton for Yarn shuffle service
2014-11-05 15:42:05 -08:00
industrial-sloth f37817b18a SPARK-4222 [CORE] use readFully in FixedLengthBinaryRecordReader
replaces the existing read() call with readFully().

Author: industrial-sloth <industrial-sloth@users.noreply.github.com>

Closes #3093 from industrial-sloth/branch-1.2-fixedLenRecRdr and squashes the following commits:

a245c8a [industrial-sloth] use readFully in FixedLengthBinaryRecordReader

(cherry picked from commit 6844e7a821)
Signed-off-by: Matei Zaharia <matei@databricks.com>
2014-11-05 15:39:16 -08:00
Kay Ousterhout a46497eecc [SPARK-3984] [SPARK-3983] Fix incorrect scheduler delay and display task deserialization time in UI
This commit fixes the scheduler delay in the UI (which previously
included things that are not scheduler delay, like time to
deserialize the task and serialize the result), and also
adds information about time to deserialize tasks to the optional
additional metrics.  Time to deserialize the task can be large relative
to task time for short jobs, and understanding when it is high can help
developers realize that they should try to reduce closure size (e.g, by including
less data in the task description).

cc shivaram etrain

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #2832 from kayousterhout/SPARK-3983 and squashes the following commits:

0c1398e [Kay Ousterhout] Fixed ordering
531575d [Kay Ousterhout] Removed executor launch time
1f13afe [Kay Ousterhout] Minor spacing fixes
335be4b [Kay Ousterhout] Made metrics hideable
5bc3cba [Kay Ousterhout] [SPARK-3984] [SPARK-3983] Improve UI task metrics.
2014-11-05 15:30:31 -08:00
Aaron Davidson 4c42986cc0 [SPARK-4242] [Core] Add SASL to external shuffle service
Does three things: (1) Adds SASL to ExternalShuffleClient, (2) puts SecurityManager in BlockManager's constructor, and (3) adds unit test.

Author: Aaron Davidson <aaron@databricks.com>

Closes #3108 from aarondav/sasl-client and squashes the following commits:

48b622d [Aaron Davidson] Screw it, let's just get LimitedInputStream
3543b70 [Aaron Davidson] Back out of pom change due to unknown test issue?
b58518a [Aaron Davidson] ByteStreams.limit() not available :(
cbe451a [Aaron Davidson] Address comments
2bf2908 [Aaron Davidson] [SPARK-4242] [Core] Add SASL to external shuffle service
2014-11-05 14:38:43 -08:00
Aaron Davidson 5e73138a01 [SPARK-2938] Support SASL authentication in NettyBlockTransferService
Also lays the groundwork for supporting it inside the external shuffle service.

Author: Aaron Davidson <aaron@databricks.com>

Closes #3087 from aarondav/sasl and squashes the following commits:

3481718 [Aaron Davidson] Delete rogue println
44f8410 [Aaron Davidson] Delete documentation - muahaha!
eb9f065 [Aaron Davidson] Improve documentation and add end-to-end test at Spark-level
a6b95f1 [Aaron Davidson] Address comments
785bbde [Aaron Davidson] Cleanup
79973cb [Aaron Davidson] Remove unused file
151b3c5 [Aaron Davidson] Add docs, timeout config, better failure handling
f6177d7 [Aaron Davidson] Cleanup SASL state upon connection termination
7b42adb [Aaron Davidson] Add unit tests
8191bcb [Aaron Davidson] [SPARK-2938] Support SASL authentication in NettyBlockTransferService
2014-11-04 16:15:38 -08:00
Davies Liu e4f42631a6 [SPARK-3886] [PySpark] simplify serializer, use AutoBatchedSerializer by default.
This PR simplify serializer, always use batched serializer (AutoBatchedSerializer as default), even batch size is 1.

Author: Davies Liu <davies@databricks.com>

This patch had conflicts when merged, resolved by
Committer: Josh Rosen <joshrosen@databricks.com>

Closes #2920 from davies/fix_autobatch and squashes the following commits:

e544ef9 [Davies Liu] revert unrelated change
6880b14 [Davies Liu] Merge branch 'master' of github.com:apache/spark into fix_autobatch
1d557fc [Davies Liu] fix tests
8180907 [Davies Liu] Merge branch 'master' of github.com:apache/spark into fix_autobatch
76abdce [Davies Liu] clean up
53fa60b [Davies Liu] Merge branch 'master' of github.com:apache/spark into fix_autobatch
d7ac751 [Davies Liu] Merge branch 'master' of github.com:apache/spark into fix_autobatch
2cc2497 [Davies Liu] Merge branch 'master' of github.com:apache/spark into fix_autobatch
b4292ce [Davies Liu] fix bug in master
d79744c [Davies Liu] recover hive tests
be37ece [Davies Liu] refactor
eb3938d [Davies Liu] refactor serializer in scala
8d77ef2 [Davies Liu] simplify serializer, use AutoBatchedSerializer by default.
2014-11-03 23:56:14 -08:00
zsxwing b671ce047d [SPARK-4166][Core] Add a backward compatibility test for ExecutorLostFailure
Author: zsxwing <zsxwing@gmail.com>

Closes #3085 from zsxwing/SPARK-4166-back-comp and squashes the following commits:

89329f4 [zsxwing] Add a backward compatibility test for ExecutorLostFailure
2014-11-03 22:47:45 -08:00
zsxwing 9bdc8412a0 [SPARK-4163][Core] Add a backward compatibility test for FetchFailed
/cc aarondav

Author: zsxwing <zsxwing@gmail.com>

Closes #3086 from zsxwing/SPARK-4163-back-comp and squashes the following commits:

21cb2a8 [zsxwing] Add a backward compatibility test for FetchFailed
2014-11-03 22:40:43 -08:00
Josh Rosen 4f035dd2cd [SPARK-611] Display executor thread dumps in web UI
This patch allows executor thread dumps to be collected on-demand and viewed in the Spark web UI.

The thread dumps are collected using Thread.getAllStackTraces().  To allow remote thread dumps to be triggered from the web UI, I added a new `ExecutorActor` that runs inside of the Executor actor system and responds to RPCs from the driver.  The driver's mechanism for obtaining a reference to this actor is a little bit hacky: it uses the block manager master actor to determine the host/port of the executor actor systems in order to construct ActorRefs to ExecutorActor.  Unfortunately, I couldn't find a much cleaner way to do this without a big refactoring of the executor -> driver communication.

Screenshots:

![image](https://cloud.githubusercontent.com/assets/50748/4781793/7e7a0776-5cbf-11e4-874d-a91cd04620bd.png)

![image](https://cloud.githubusercontent.com/assets/50748/4781794/8bce76aa-5cbf-11e4-8d13-8477748c9f7e.png)

![image](https://cloud.githubusercontent.com/assets/50748/4781797/bd11a8b8-5cbf-11e4-9ad7-a7459467ec8e.png)

Author: Josh Rosen <joshrosen@databricks.com>

Closes #2944 from JoshRosen/jstack-in-web-ui and squashes the following commits:

3c21a5d [Josh Rosen] Address review comments:
880f7f7 [Josh Rosen] Merge remote-tracking branch 'origin/master' into jstack-in-web-ui
f719266 [Josh Rosen] Merge remote-tracking branch 'origin/master' into jstack-in-web-ui
19707b0 [Josh Rosen] Add one comment.
127a130 [Josh Rosen] Update to use SparkContext.DRIVER_IDENTIFIER
b8e69aa [Josh Rosen] Merge remote-tracking branch 'origin/master' into jstack-in-web-ui
3dfc2d4 [Josh Rosen] Add missing file.
bc1e675 [Josh Rosen] Undo some leftover changes from the earlier approach.
f4ac1c1 [Josh Rosen] Switch to on-demand collection of thread dumps
dfec08b [Josh Rosen] Add option to disable thread dumps in UI.
4c87d7f [Josh Rosen] Use separate RPC for sending thread dumps.
2b8bdf3 [Josh Rosen] Enable thread dumps from the driver when running in non-local mode.
cc3e6b3 [Josh Rosen] Fix test code in DAGSchedulerSuite.
87b8b65 [Josh Rosen] Add new listener event for thread dumps.
8c10216 [Josh Rosen] Add missing file.
0f198ac [Josh Rosen] [SPARK-611] Display executor thread dumps in web UI
2014-11-03 18:18:47 -08:00
Zhang, Liye 97a466eca0 [SPARK-4168][WebUI] web statges number should show correctly when stages are more than 1000
The number of completed stages and failed stages showed on webUI will always be less than 1000. This is really misleading when there are already thousands of stages completed or failed. The number should be correct even when only partial stages listed on the webUI (stage info will be removed if the number is too large).

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

Closes #3035 from liyezhang556520/webStageNum and squashes the following commits:

d9e29fb [Zhang, Liye] add detailed comments for variables
4ea8fd1 [Zhang, Liye] change variable name accroding to comments
f4c404d [Zhang, Liye] [SPARK-4168][WebUI] web statges number should show correctly when stages are more than 1000
2014-11-03 18:17:32 -08:00
Sandy Ryza 28128150e7 SPARK-4178. Hadoop input metrics ignore bytes read in RecordReader insta...
...ntiation

Author: Sandy Ryza <sandy@cloudera.com>

Closes #3045 from sryza/sandy-spark-4178 and squashes the following commits:

8d2e70e [Sandy Ryza] Kostas's review feedback
e5b27c0 [Sandy Ryza] SPARK-4178. Hadoop input metrics ignore bytes read in RecordReader instantiation
2014-11-03 15:19:01 -08:00
zsxwing 76386e1a23 [SPARK-4163][Core][WebUI] Send the fetch failure message back to Web UI
This is a PR to send the fetch failure message back to Web UI.
Before:
![f1](https://cloud.githubusercontent.com/assets/1000778/4856595/1f036c80-60be-11e4-956f-335147fbccb7.png)
![f2](https://cloud.githubusercontent.com/assets/1000778/4856596/1f11cbea-60be-11e4-8fe9-9f9b2b35c884.png)

After (Please ignore the meaning of exception, I threw it in the code directly because it's hard to simulate a fetch failure):
![e1](https://cloud.githubusercontent.com/assets/1000778/4856600/2657ea38-60be-11e4-9f2d-d56c5f900f10.png)
![e2](https://cloud.githubusercontent.com/assets/1000778/4856601/26595008-60be-11e4-912b-2744af786991.png)

Author: zsxwing <zsxwing@gmail.com>

Closes #3032 from zsxwing/SPARK-4163 and squashes the following commits:

f7e1faf [zsxwing] Discard changes for FetchFailedException and minor modification
4e946f7 [zsxwing] Add e as the cause of SparkException
316767d [zsxwing] Add private[storage] to FetchResult
d51b0b6 [zsxwing] Set e as the cause of FetchFailedException
b88c919 [zsxwing] Use 'private[storage]' for case classes instead of 'sealed'
62103fd [zsxwing] Update as per review
0c07d1f [zsxwing] Backward-compatible support
a3bca65 [zsxwing] Send the fetch failure message back to Web UI
2014-11-02 23:20:22 -08:00
Aaron Davidson 2ebd1df3f1 [SPARK-4183] Close transport-related resources between SparkContexts
A leak of event loops may be causing test failures.

Author: Aaron Davidson <aaron@databricks.com>

Closes #3053 from aarondav/leak and squashes the following commits:

e676d18 [Aaron Davidson] Typo!
8f96475 [Aaron Davidson] Keep original ssc semantics
7e49f10 [Aaron Davidson] A leak of event loops may be causing test failures.
2014-11-02 16:26:24 -08:00
Cheng Lian c9f840046f [SPARK-3791][SQL] Provides Spark version and Hive version in HiveThriftServer2
This PR overrides the `GetInfo` Hive Thrift API to provide correct version information. Another property `spark.sql.hive.version` is added to reveal the underlying Hive version. These are generally useful for Spark SQL ODBC driver providers. The Spark version information is extracted from the jar manifest. Also took the chance to remove the `SET -v` hack, which was a workaround for Simba ODBC driver connectivity.

TODO

- [x] Find a general way to figure out Hive (or even any dependency) version.

  This [blog post](http://blog.soebes.de/blog/2014/01/02/version-information-into-your-appas-with-maven/) suggests several methods to inspect application version. In the case of Spark, this can be tricky because the chosen method:

  1. must applies to both Maven build and SBT build

    For Maven builds, we can retrieve the version information from the META-INF/maven directory within the assembly jar. But this doesn't work for SBT builds.

  2. must not rely on the original jars of dependencies to extract specific dependency version, because Spark uses assembly jar.

    This implies we can't read Hive version from Hive jar files since standard Spark distribution doesn't include them.

  3. should play well with `SPARK_PREPEND_CLASSES` to ease local testing during development.

     `SPARK_PREPEND_CLASSES` prevents classes to be loaded from the assembly jar, thus we can't locate the jar file and read its manifest.

  Given these, maybe the only reliable method is to generate a source file containing version information at build time. pwendell Do you have any suggestions from the perspective of the build process?

**Update** Hive version is now retrieved from the newly introduced `HiveShim` object.

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

Closes #2843 from liancheng/get-info and squashes the following commits:

a873d0f [Cheng Lian] Updates test case
53f43cd [Cheng Lian] Retrieves underlying Hive verson via HiveShim
1d282b8 [Cheng Lian] Removes the Simba ODBC "SET -v" hack
f857fce [Cheng Lian] Overrides Hive GetInfo Thrift API and adds Hive version property
2014-11-02 15:18:29 -08:00
zsxwing 4e6a7a0b3e [SPARK-4166][Core][WebUI] Display the executor ID in the Web UI when ExecutorLostFailure happens
Now when ExecutorLostFailure happens, it only displays `ExecutorLostFailure (executor lost)`. Adding the executor id will help locate the faulted executor.

Author: zsxwing <zsxwing@gmail.com>

Closes #3033 from zsxwing/SPARK-4166 and squashes the following commits:

ff4664c [zsxwing] Backward-compatible support
c5c4cf2 [zsxwing] Display the executor ID in the Web UI when ExecutorLostFailure happens
2014-11-02 10:44:52 -08:00
Davies Liu 6181577e99 [SPARK-3466] Limit size of results that a driver collects for each action
Right now, operations like collect() and take() can crash the driver with an OOM if they bring back too many data.

This PR will introduce spark.driver.maxResultSize, after setting it, the driver will abort a job if its result is bigger than it.

By default, it's 1g (for backward compatibility for most the cases).

In local mode, the driver and executor share the same JVM, the default setting can not protect JVM from OOM.

cc mateiz

Author: Davies Liu <davies@databricks.com>

Closes #3003 from davies/collect and squashes the following commits:

248ed5e [Davies Liu] fix compile
272522e [Davies Liu] address comments
2c35773 [Davies Liu] add sizes in message of abort()
5d62303 [Davies Liu] address comments
bc3c077 [Davies Liu] Merge branch 'master' of github.com:apache/spark into collect
11f97c5 [Davies Liu] address comments
47b144f [Davies Liu] check the size of result before send and fetch
3d81af2 [Davies Liu] address comments
ca8267d [Davies Liu] limit the size of data by collect
2014-11-02 00:03:51 -07:00
Patrick Wendell 7894de276b Revert "[SPARK-4183] Enable NettyBlockTransferService by default"
This reverts commit 59e626c701.
2014-11-01 15:18:58 -07:00
Aaron Davidson f55218aeb1 [SPARK-3796] Create external service which can serve shuffle files
This patch introduces the tooling necessary to construct an external shuffle service which is independent of Spark executors, and then use this service inside Spark. An example (just for the sake of this PR) of the service creation can be found in Worker, and the service itself is used by plugging in the StandaloneShuffleClient as Spark's ShuffleClient (setup in BlockManager).

This PR continues the work from #2753, which extracted out the transport layer of Spark's block transfer into an independent package within Spark. A new package was created which contains the Spark business logic necessary to retrieve the actual shuffle data, which is completely independent of the transport layer introduced in the previous patch. Similar to the transport layer, this package must not depend on Spark as we anticipate plugging this service as a lightweight process within, say, the YARN NodeManager, and do not wish to include Spark's dependencies (including Scala itself).

There are several outstanding tasks which must be complete before this PR can be merged:
- [x] Complete unit testing of network/shuffle package.
- [x] Performance and correctness testing on a real cluster.
- [x] Remove example service instantiation from Worker.scala.

There are even more shortcomings of this PR which should be addressed in followup patches:
- Don't use Java serializer for RPC layer! It is not cross-version compatible.
- Handle shuffle file cleanup for dead executors once the application terminates or the ContextCleaner triggers.
- Documentation of the feature in the Spark docs.
- Improve behavior if the shuffle service itself goes down (right now we don't blacklist it, and new executors cannot spawn on that machine).
- SSL and SASL integration
- Nice to have: Handle shuffle file consolidation (this would requires changes to Spark's implementation).

Author: Aaron Davidson <aaron@databricks.com>

Closes #3001 from aarondav/shuffle-service and squashes the following commits:

4d1f8c1 [Aaron Davidson] Remove changes to Worker
705748f [Aaron Davidson] Rename Standalone* to External*
fd3928b [Aaron Davidson] Do not unregister executor outputs unduly
9883918 [Aaron Davidson] Make suggested build changes
3d62679 [Aaron Davidson] Add Spark integration test
7fe51d5 [Aaron Davidson] Fix SBT integration
56caa50 [Aaron Davidson] Address comments
c8d1ac3 [Aaron Davidson] Add unit tests
2f70c0c [Aaron Davidson] Fix unit tests
5483e96 [Aaron Davidson] Fix unit tests
46a70bf [Aaron Davidson] Whoops, bracket
5ea4df6 [Aaron Davidson] [SPARK-3796] Create external service which can serve shuffle files
2014-11-01 14:37:45 -07:00
Aaron Davidson 59e626c701 [SPARK-4183] Enable NettyBlockTransferService by default
Note that we're turning this on for at least the first part of the QA period as a trial. We want to enable this (and deprecate the NioBlockTransferService) as soon as possible in the hopes that NettyBlockTransferService will be more stable and easier to maintain. We will turn it off if we run into major issues.

Author: Aaron Davidson <aaron@databricks.com>

Closes #3049 from aarondav/enable-netty and squashes the following commits:

bb981cc [Aaron Davidson] [SPARK-4183] Enable NettyBlockTransferService by default
2014-11-01 13:15:24 -07:00
Kevin Mader 7136719b7d [SPARK-2759][CORE] Generic Binary File Support in Spark
The additions add the abstract BinaryFileInputFormat and BinaryRecordReader classes for reading in data as a byte stream and converting it to another format using the ```def parseByteArray(inArray: Array[Byte]): T``` function.
As a trivial example ```ByteInputFormat``` and ```ByteRecordReader``` are included which just return the Array[Byte] from a given file.
Finally a RDD for ```BinaryFileInputFormat``` (to allow for easier partitioning changes as was done for WholeFileInput) was added and the appropriate byteFiles to the ```SparkContext``` so the functions can be easily used by others.
A common use case might be to read in a folder
```
sc.byteFiles("s3://mydrive/tif/*.tif").map(rawData => ReadTiffFromByteArray(rawData))
```

Author: Kevin Mader <kevinmader@gmail.com>
Author: Kevin Mader <kmader@users.noreply.github.com>

Closes #1658 from kmader/master and squashes the following commits:

3c49a30 [Kevin Mader] fixing wholetextfileinput to it has the same setMinPartitions function as in BinaryData files
359a096 [Kevin Mader] making the final corrections suggested by @mateiz and renaming a few functions to make their usage clearer
6379be4 [Kevin Mader] reorganizing code
7b9d181 [Kevin Mader] removing developer API, cleaning up imports
8ac288b [Kevin Mader] fixed a single slightly over 100 character line
92bda0d [Kevin Mader] added new tests, renamed files, fixed several of the javaapi functions, formatted code more nicely
a32fef7 [Kevin Mader] removed unneeded classes added DeveloperApi note to portabledatastreams since the implementation might change
49174d9 [Kevin Mader] removed unneeded classes added DeveloperApi note to portabledatastreams since the implementation might change
c27a8f1 [Kevin Mader] jenkins crashed before running anything last time, so making minor change
b348ce1 [Kevin Mader] fixed order in check (prefix only appears on jenkins not when I run unit tests locally)
0588737 [Kevin Mader] filename check in "binary file input as byte array" test now ignores prefixes and suffixes which might get added by Hadoop
4163e38 [Kevin Mader] fixing line length and output from FSDataInputStream to DataInputStream to minimize sensitivity to Hadoop API changes
19812a8 [Kevin Mader] Fixed the serialization issue with PortableDataStream since neither CombineFileSplit nor TaskAttemptContext implement the Serializable interface, by using ByteArrays for storing both and then recreating the objects from these bytearrays as needed.
238c83c [Kevin Mader] fixed several scala-style issues, changed structure of binaryFiles, removed excessive classes added new tests. The caching tests still have a serialization issue, but that should be easily fixed as well.
932a206 [Kevin Mader] Update RawFileInput.scala
a01c9cf [Kevin Mader] Update RawFileInput.scala
441f79a [Kevin Mader] fixed a few small comments and dependency
12e7be1 [Kevin Mader] removing imglib from maven (definitely not ready yet)
5deb79e [Kevin Mader] added new portabledatastream to code so that it can be serialized correctly
f032bc0 [Kevin Mader] fixed bug in path name, renamed tests
bc5c0b9 [Kevin Mader] made minor stylistic adjustments from mateiz
df8e528 [Kevin Mader] fixed line lengths and changed java test
9a313d5 [Kevin Mader] making classes that needn't be public private, adding automatic file closure, adding new tests
edf5829 [Kevin Mader] fixing line lengths, adding new lines
f4841dc [Kevin Mader] un-optimizing imports, silly intellij
eacfaa6 [Kevin Mader] Added FixedLengthBinaryInputFormat and RecordReader from freeman-lab and added them to both the JavaSparkContext and the SparkContext as fixedLengthBinaryFile
1622935 [Kevin Mader] changing the line lengths to make jenkins happy
1cfa38a [Kevin Mader] added apache headers, added datainputstream directly as an output option for more complicated readers (HDF5 perhaps), and renamed several of the functions and files to be more consistent. Also added parallel functions to the java api
84035f1 [Kevin Mader] adding binary and byte file support spark
81c5f12 [Kevin Mader] Merge pull request #1 from apache/master
2014-11-01 11:59:39 -07:00
Sandy Ryza 23f73f525c SPARK-4175. Exception on stage page
Author: Sandy Ryza <sandy@cloudera.com>

Closes #3043 from sryza/sandy-spark-4175 and squashes the following commits:

e327340 [Sandy Ryza] SPARK-4175. Exception on stage page
2014-10-31 17:22:52 -07:00
Mark Mims a68ecf3281 [SPARK-4141] Hide Accumulators column on stage page when no accumulators exist
WebUI

Author: Mark Mims <mark.mims@canonical.com>

This patch had conflicts when merged, resolved by
Committer: Josh Rosen <joshrosen@databricks.com>

Closes #3031 from mmm/remove-accumulators-col and squashes the following commits:

6141cb3 [Mark Mims] reformat to satisfy scalastyle linelength.  build failed from jenkins https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22604/
390893b [Mark Mims] cleanup
c28c449 [Mark Mims] looking much better now... minimal explicit formatting.  Now, see if any sort keys make sense
fb72156 [Mark Mims] mimic hasInput.  The basics work here, but wanna clean this up with maybeAccumulators for column content
2014-10-31 11:41:03 -07:00
Kay Ousterhout adb6415c1d [SPARK-4016] Allow user to show/hide UI metrics.
This commit adds a set of checkboxes to the stage detail
page that the user can use to show additional task metrics,
including the GC time, result serialization time, result fetch
time, and scheduler delay.  All of these metrics are now
hidden by default.  This allows advanced users to look at more
detailed metrics, without distracting the average user.

This change also cleans up the stage detail page so that metrics
are shown in the same order in the summary table as in the task table,
and updates the metrics in both tables such that they contain the same
set of metrics.

The ability to remember a user's preferences for which metrics
should be shown has been filed as SPARK-4024.

Here's what the stage detail page looks like by default:
![image](https://cloud.githubusercontent.com/assets/1108612/4744322/3ebe319e-5a2f-11e4-891f-c792be79caa2.png)

and once a user clicks "Show additional metrics" (note that all the metrics get checked by default):
![image](https://cloud.githubusercontent.com/assets/1108612/4744332/51e5abda-5a2f-11e4-8994-d0d3705ee05d.png)

cc shivaram andrewor14

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #2867 from kayousterhout/SPARK-checkboxes and squashes the following commits:

6015913 [Kay Ousterhout] Added comment
08dee73 [Kay Ousterhout] Josh's usability comments
0940d61 [Kay Ousterhout] Style updates based on Andrew's review
ef05ccd [Kay Ousterhout] Added tooltips
d7cfaaf [Kay Ousterhout] Made list of add'l metrics collapsible.
70c1fb5 [Kay Ousterhout] [SPARK-4016] Allow user to show/hide UI metrics.
2014-10-31 10:28:19 -07:00
Erik Erlandson ad3bd0dff8 [SPARK-3250] Implement Gap Sampling optimization for random sampling
More efficient sampling, based on Gap Sampling optimization:
http://erikerlandson.github.io/blog/2014/09/11/faster-random-samples-with-gap-sampling/

Author: Erik Erlandson <eerlands@redhat.com>

Closes #2455 from erikerlandson/spark-3250-pr and squashes the following commits:

72496bc [Erik Erlandson] [SPARK-3250] Implement Gap Sampling optimization for random sampling
2014-10-30 22:30:52 -07:00
Patrick Wendell 0734d09320 HOTFIX: Clean up build in network module.
This is currently breaking the package build for some people (including me).

This patch does some general clean-up which also fixes the current issue.
- Uses consistent artifact naming
- Adds sbt support for this module
- Changes tests to use scalatest (fixes the original issue[1])

One thing to note, it turns out that scalatest when invoked in the
Maven build doesn't succesfully detect JUnit Java tests. This is
a long standing issue, I noticed it applies to all of our current
test suites as well. I've created SPARK-4159 to fix this.

[1] The original issue is that we need to allocate extra memory
for the tests, happens by default in our scalatest configuration.

Author: Patrick Wendell <pwendell@gmail.com>

Closes #3025 from pwendell/hotfix and squashes the following commits:

faa9053 [Patrick Wendell] HOTFIX: Clean up build in network module.
2014-10-30 20:15:36 -07:00
Andrew Or 26d31d15fd Revert "SPARK-1209 [CORE] SparkHadoop{MapRed,MapReduce}Util should not use package org.apache.hadoop"
This reverts commit 68cb69daf3.
2014-10-30 17:56:10 -07:00
Sean Owen 68cb69daf3 SPARK-1209 [CORE] SparkHadoop{MapRed,MapReduce}Util should not use package org.apache.hadoop
(This is just a look at what completely moving the classes would look like. I know Patrick flagged that as maybe not OK, although, it's private?)

Author: Sean Owen <sowen@cloudera.com>

Closes #2814 from srowen/SPARK-1209 and squashes the following commits:

ead1115 [Sean Owen] Disable MIMA warnings resulting from moving the class -- this was also part of the PairRDDFunctions type hierarchy though?
2d42c1d [Sean Owen] Move SparkHadoopMapRedUtil / SparkHadoopMapReduceUtil from org.apache.hadoop to org.apache.spark
2014-10-30 15:54:53 -07:00
Andrew Or 2f54543815 [SPARK-3661] Respect spark.*.memory in cluster mode
This also includes minor re-organization of the code. Tested locally in both client and deploy modes.

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

Closes #2697 from andrewor14/memory-cluster-mode and squashes the following commits:

01d78bc [Andrew Or] Merge branch 'master' of github.com:apache/spark into memory-cluster-mode
ccd468b [Andrew Or] Add some comments per Patrick
c956577 [Andrew Or] Tweak wording
2b4afa0 [Andrew Or] Unused import
47a5a88 [Andrew Or] Correct Spark properties precedence order
bf64717 [Andrew Or] Merge branch 'master' of github.com:apache/spark into memory-cluster-mode
dd452d0 [Andrew Or] Respect spark.*.memory in cluster mode
2014-10-30 15:44:29 -07:00
zsxwing d345057835 [SPARK-4153][WebUI] Update the sort keys for HistoryPage
Sort "Started", "Completed", "Duration" and "Last Updated" by time.

Author: zsxwing <zsxwing@gmail.com>

Closes #3014 from zsxwing/SPARK-4153 and squashes the following commits:

ec8b9ad [zsxwing] Sort "Started", "Completed", "Duration" and "Last Updated" by time
2014-10-30 15:33:56 -07:00
Andrew Or 849b43ec0f Minor style hot fix after #2711
I had planned to fix this when I merged it but I forgot to. witgo

Author: Andrew Or <andrew@databricks.com>

Closes #3018 from andrewor14/command-utils-style and squashes the following commits:

c2959fb [Andrew Or] Style hot fix
2014-10-30 15:33:34 -07:00
Andrew Or 9334d69967 [SPARK-4155] Consolidate usages of <driver>
We use "\<driver\>" everywhere. Let's not do that.

Author: Andrew Or <andrew@databricks.com>

Closes #3020 from andrewor14/consolidate-driver and squashes the following commits:

c1c2204 [Andrew Or] Just use "<driver>" for local executor ID
3d751e9 [Andrew Or] Consolidate usages of <driver>
2014-10-30 15:32:46 -07:00
Andrew Or 5231a3f228 [Minor] A few typos in comments and log messages
Author: Andrew Or <andrewor14@gmail.com>
Author: Andrew Or <andrew@databricks.com>

Closes #3021 from andrewor14/typos and squashes the following commits:

daaf417 [Andrew Or] Merge branch 'master' of github.com:apache/spark into typos
4838ae4 [Andrew Or] Merge branch 'master' of github.com:apache/spark into typos
026d426 [Andrew Or] Merge branch 'master' of github.com:andrewor14/spark into typos
a81ae8f [Andrew Or] Some typos
2014-10-30 15:32:11 -07:00
Andrew Or 24c5129257 [SPARK-3319] [SPARK-3338] Resolve Spark submit config paths
The bulk of this PR is comprised of tests. All changes in functionality are made in `SparkSubmit.scala` (~20 lines).

**SPARK-3319.** There is currently a divergence in behavior when the user passes in additional jars through `--jars` and through setting `spark.jars` in the default properties file. The former will happily resolve the paths (e.g. convert `my.jar` to `file:/absolute/path/to/my.jar`), while the latter does not. We should resolve paths consistently in both cases. This also applies to the following pairs of command line arguments and Spark configs:

- `--jars` ~ `spark.jars`
- `--files` ~ `spark.files` / `spark.yarn.dist.files`
- `--archives` ~ `spark.yarn.dist.archives`
- `--py-files` ~ `spark.submit.pyFiles`

**SPARK-3338.** This PR also fixes the following bug: if the user sets `spark.submit.pyFiles` in his/her properties file, it does not actually get picked up even if `--py-files` is not set. This is simply because the config is overridden by an empty string.

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

Closes #2232 from andrewor14/resolve-config-paths and squashes the following commits:

fff2869 [Andrew Or] Add spark.yarn.jar
da3a1c1 [Andrew Or] Merge branch 'master' of github.com:apache/spark into resolve-config-paths
f0fae64 [Andrew Or] Merge branch 'master' of github.com:apache/spark into resolve-config-paths
05e03d6 [Andrew Or] Add tests for resolving both command line and config paths
460117e [Andrew Or] Resolve config paths properly
fe039d3 [Andrew Or] Beef up tests to test fixed-pointed-ness of Utils.resolveURI(s)
2014-10-30 15:29:07 -07:00
Grace 9142c9b80b [SPARK-4078] New FsPermission instance w/o FsPermission.createImmutable in eventlog
By default, Spark builds its package against Hadoop 1.0.4 version. In that version, it has some FsPermission bug (see [HADOOP-7629] (https://issues.apache.org/jira/browse/HADOOP-7629) by Todd Lipcon). This bug got fixed since 1.1 version. By using that FsPermission.createImmutable() API, end-user may see some RPC exception like below (if turn on eventlog over HDFS).  Here proposes a quick fix to avoid certain exception for all hadoop versions.
```
Exception in thread "main" java.io.IOException: Call to sr484/10.1.2.84:54310 failed on local exception: java.io.EOFException
        at org.apache.hadoop.ipc.Client.wrapException(Client.java:1150)
        at org.apache.hadoop.ipc.Client.call(Client.java:1118)
        at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:229)
        at $Proxy6.setPermission(Unknown Source)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
        at java.lang.reflect.Method.invoke(Method.java:597)
        at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:85)
        at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:62)
        at $Proxy6.setPermission(Unknown Source)
        at org.apache.hadoop.hdfs.DFSClient.setPermission(DFSClient.java:1285)
        at org.apache.hadoop.hdfs.DistributedFileSystem.setPermission(DistributedFileSystem.java:572)
        at org.apache.spark.util.FileLogger.createLogDir(FileLogger.scala:138)
        at org.apache.spark.util.FileLogger.start(FileLogger.scala:115)
        at org.apache.spark.scheduler.EventLoggingListener.start(EventLoggingListener.scala:74)
        at org.apache.spark.SparkContext.<init>(SparkContext.scala:324)
```

Author: Grace <jie.huang@intel.com>

Closes #2892 from GraceH/eventlog-rpc and squashes the following commits:

58ea038 [Grace] new FsPermission Instance w/o FsPermission.createImmutable
2014-10-30 15:27:32 -07:00
Tathagata Das fb1fbca204 [SPARK-4027][Streaming] WriteAheadLogBackedBlockRDD to read received either from BlockManager or WAL in HDFS
As part of the initiative of preventing data loss on streaming driver failure, this sub-task implements a BlockRDD that is backed by HDFS. This BlockRDD can either read data from the Spark's BlockManager, or read the data from file-segments in write ahead log in HDFS.

Most of this code has been written by @harishreedharan

Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Hari Shreedharan <hshreedharan@apache.org>

Closes #2931 from tdas/driver-ha-rdd and squashes the following commits:

209e49c [Tathagata Das] Better fix to style issue.
4a5866f [Tathagata Das] Addressed one more comment.
ed5fbf0 [Tathagata Das] Minor updates.
b0a18b1 [Tathagata Das] Fixed import order.
20aa7c6 [Tathagata Das] Fixed more line length issues.
29aa099 [Tathagata Das] Fixed line length issues.
9e47b5b [Tathagata Das] Renamed class, simplified+added unit tests.
6e1bfb8 [Tathagata Das] Tweaks testuite to create spark contxt lazily to prevent contxt leaks.
9c86a61 [Tathagata Das] Merge pull request #22 from harishreedharan/driver-ha-rdd
2878c38 [Hari Shreedharan] Shutdown spark context after tests. Formatting/minor fixes
c709f2f [Tathagata Das] Merge pull request #21 from harishreedharan/driver-ha-rdd
5cce16f [Hari Shreedharan] Make sure getBlockLocations uses offset and length to find the blocks on HDFS
eadde56 [Tathagata Das] Transferred HDFSBackedBlockRDD for the driver-ha-working branch
2014-10-30 15:17:02 -07:00
Kay Ousterhout 6db3157464 [SPARK-4102] Remove unused ShuffleReader.stop() method.
This method is not implemented by the only subclass
(HashShuffleReader), nor is it ever called. While the
use of Scala's fancy "???" was pretty exciting, the method's
existence can only lead to confusion and it therefore should
be deleted.

mateiz was there a reason for adding this that I'm
missing?

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #2966 from kayousterhout/SPARK-4102 and squashes the following commits:

532c564 [Kay Ousterhout] Added back commented-out method, as per Matei's request
904655e [Kay Ousterhout] [SPARK-4102] Remove unused ShuffleReader.stop() method.
2014-10-29 23:52:46 -07:00
GuoQiang Li cd739bd756 [SPARK-1720][SPARK-1719] use LD_LIBRARY_PATH instead of -Djava.library.path
- [X] Standalone
- [X] YARN
- [X] Mesos
- [X]  Mac OS X
- [X] Linux
- [ ]  Windows

This is another implementation about #1031

Author: GuoQiang Li <witgo@qq.com>

Closes #2711 from witgo/SPARK-1719 and squashes the following commits:

c7b26f6 [GuoQiang Li] review commits
4488e41 [GuoQiang Li] Refactoring CommandUtils
a444094 [GuoQiang Li] review commits
40c0b4a [GuoQiang Li] Add buildLocalCommand method
c1a0ddd [GuoQiang Li] fix comments
156ce88 [GuoQiang Li] review commit
38aa377 [GuoQiang Li] Refactor CommandUtils.scala
4269e00 [GuoQiang Li] Refactor SparkSubmitDriverBootstrapper.scala
7a1d634 [GuoQiang Li] use LD_LIBRARY_PATH instead of -Djava.library.path
2014-10-29 23:02:58 -07:00
Andrew Or 8d59b37b02 [SPARK-3795] Heuristics for dynamically scaling executors
This is part of a bigger effort to provide elastic scaling of executors within a Spark application ([SPARK-3174](https://issues.apache.org/jira/browse/SPARK-3174)). This PR does not provide any functionality by itself; it is a skeleton that is missing a mechanism to be added later in [SPARK-3822](https://issues.apache.org/jira/browse/SPARK-3822).

Comments and feedback are most welcome. For those of you reviewing this in detail, I highly recommend doing it through your favorite IDE instead of through the diff here.

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

Closes #2746 from andrewor14/scaling-heuristics and squashes the following commits:

8a4fdaa [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics
e045df8 [Andrew Or] Add warning message (minor)
dfa31ec [Andrew Or] Fix tests
c0becc4 [Andrew Or] Merging with SPARK-3822
4784f93 [Andrew Or] Reword an awkward log message
181f27f [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics
c79e907 [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics
4672b90 [Andrew Or] It's nano time.
a6a30f2 [Andrew Or] Do not allow min/max executors of 0
c60ec33 [Andrew Or] Rewrite test logic with clocks
b00b680 [Andrew Or] Fix style
c3caa65 [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics
7f9da14 [Andrew Or] Factor out logic to verify bounds on # executors (minor)
f279019 [Andrew Or] Add time mocking tests for polling loop
685e347 [Andrew Or] Factor out clock in polling loop to facilitate testing
3cea7f7 [Andrew Or] Use PrivateMethodTester to keep original class private
3156d81 [Andrew Or] Update comments and exception messages
92f36f9 [Andrew Or] Address minor review comments
abdea61 [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics
2aefd09 [Andrew Or] Correct listener behavior
9fe6e44 [Andrew Or] Rename variables and configs + update comments and log messages
149cc32 [Andrew Or] Fix style
254c958 [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics
5ff829b [Andrew Or] Add tests for ExecutorAllocationManager
19c6c4b [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics
5896515 [Andrew Or] Move ExecutorAllocationManager out of scheduler package
9ca8945 [Andrew Or] Rewrite callbacks through the listener interface
5e336b9 [Andrew Or] Remove code from backend to avoid conflict with SPARK-3822
092d1fd [Andrew Or] Remove timeout logic for pending requests
1309fab [Andrew Or] Request executors by specifying the number pending
8bc0e9d [Andrew Or] Add logic to expire pending requests after timeouts
b750ee1 [Andrew Or] Express timers in terms of expiration times + remove retry logic
7f8dd47 [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics
9d516cc [Andrew Or] Bug fix: Actually trigger the add timer / add retry timer
44f1832 [Andrew Or] Rename configs to include time units
eaae7ef [Andrew Or] Address various review comments
6f8be6c [Andrew Or] Beef up comments on what each of the timers mean
baaa403 [Andrew Or] Simplify variable names (minor)
42beec8 [Andrew Or] Reset whether the add threshold is crossed on cancellation
9bcc0bc [Andrew Or] ExecutorScalingManager -> ExecutorAllocationManager
2784398 [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics
5a97d9e [Andrew Or] Log retry attempts in INFO + clean up logging
2f55c9f [Andrew Or] Do not keep requesting executors even after max attempts
0acd1cb [Andrew Or] Rewrite timer logic with polling
b3c7d44 [Andrew Or] Start the retry timer for adding executors at the right time
9b5f2ea [Andrew Or] Wording changes in comments and log messages
c2203a5 [Andrew Or] Simplify code to access the scheduler backend
e519d08 [Andrew Or] Simplify initialization code
2cc87a7 [Andrew Or] Add retry logic for removing executors
d0b34a6 [Andrew Or] Add retry logic for adding executors
9cc4649 [Andrew Or] Simplifying synchronization logic
67c03c7 [Andrew Or] Correct semantics of adding executors + update comments
6c48ab0 [Andrew Or] Update synchronization comment
8901900 [Andrew Or] Simplify remove policy + change the semantics of add policy
1cc8444 [Andrew Or] Minor wording change
ae5b64a [Andrew Or] Add synchronization
20ec6b9 [Andrew Or] First cut implementation of removing executors dynamically
4077ae2 [Andrew Or] Minor code re-organization
6f1fa66 [Andrew Or] First cut implementation of adding executors dynamically
b2e6dcc [Andrew Or] Add skeleton interface for requesting / killing executors
2014-10-29 17:48:59 -07:00
zsxwing e7fd80413d [SPARK-4097] Fix the race condition of 'thread'
There is a chance that `thread` is null when calling `thread.interrupt()`.

```Scala
  override def cancel(): Unit = this.synchronized {
    _cancelled = true
    if (thread != null) {
      thread.interrupt()
    }
  }
```
Should put `thread = null` into a `synchronized` block to fix the race condition.

Author: zsxwing <zsxwing@gmail.com>

Closes #2957 from zsxwing/SPARK-4097 and squashes the following commits:

edf0aee [zsxwing] Add comments to explain the lock
c5cfeca [zsxwing] Fix the race condition of 'thread'
2014-10-29 14:42:50 -07:00
Andrew Or 1df05a40eb [SPARK-3822] Executor scaling mechanism for Yarn
This is part of a broader effort to enable dynamic scaling of executors ([SPARK-3174](https://issues.apache.org/jira/browse/SPARK-3174)). This is intended to work alongside SPARK-3795 (#2746), SPARK-3796 and SPARK-3797, but is functionally independently of these other issues.

The logic is built on top of PraveenSeluka's changes at #2798. This is different from the changes there in a few major ways: (1) the mechanism is implemented within the existing scheduler backend framework rather than in new `Actor` classes. This also introduces a parent abstract class `YarnSchedulerBackend` to encapsulate common logic to communicate with the Yarn `ApplicationMaster`. (2) The interface of requesting executors exposed to the `SparkContext` is the same, but the communication between the scheduler backend and the AM uses total number executors desired instead of an incremental number. This is discussed in #2746 and explained in the comments in the code.

I have tested this significantly on a stable Yarn cluster.

------------
A remaining task for this issue is to tone down the error messages emitted when an executor is removed.
Currently, `SparkContext` and its components react as if the executor has failed, resulting in many scary error messages and eventual timeouts. While it's not strictly necessary to fix this as of the first-cut implementation of this mechanism, it would be good to add logic to distinguish this case. I prefer to address this in a separate PR. I have filed a separate JIRA for this task at SPARK-4134.

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

Closes #2840 from andrewor14/yarn-scaling-mechanism and squashes the following commits:

485863e [Andrew Or] Minor log message changes
4920be8 [Andrew Or] Clarify that public API is only for Yarn mode for now
1c57804 [Andrew Or] Reword a few comments + other review comments
6321140 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-scaling-mechanism
02836c0 [Andrew Or] Limit scope of synchronization
4e2ed7f [Andrew Or] Fix bug: keep track of removed executors properly
73ade46 [Andrew Or] Wording changes (minor)
2a7a6da [Andrew Or] Add `sc.killExecutor` as a shorthand (minor)
665f229 [Andrew Or] Mima excludes
79aa2df [Andrew Or] Simplify the request interface by asking for a total
04f625b [Andrew Or] Fix race condition that causes over-allocation of executors
f4783f8 [Andrew Or] Change the semantics of requesting executors
005a124 [Andrew Or] Fix tests
4628b16 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-scaling-mechanism
db4a679 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-scaling-mechanism
572f5c5 [Andrew Or] Unused import (minor)
f30261c [Andrew Or] Kill multiple executors rather than one at a time
de260d9 [Andrew Or] Simplify by skipping useless null check
9c52542 [Andrew Or] Simplify by skipping the TaskSchedulerImpl
97dd1a8 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-scaling-mechanism
d987b3e [Andrew Or] Move addWebUIFilters to Yarn scheduler backend
7b76d0a [Andrew Or] Expose mechanism in SparkContext as developer API
47466cd [Andrew Or] Refactor common Yarn scheduler backend logic
c4dfaac [Andrew Or] Avoid thrashing when removing executors
53e8145 [Andrew Or] Start yarn actor early to listen for AM registration message
bbee669 [Andrew Or] Add mechanism in yarn client mode
2014-10-29 14:01:00 -07:00
Reynold Xin dff015533d [SPARK-3453] Netty-based BlockTransferService, extracted from Spark core
This PR encapsulates #2330, which is itself a continuation of #2240. The first goal of this PR is to provide an alternate, simpler implementation of the ConnectionManager which is based on Netty.

In addition to this goal, however, we want to resolve [SPARK-3796](https://issues.apache.org/jira/browse/SPARK-3796), which calls for a standalone shuffle service which can be integrated into the YARN NodeManager, Standalone Worker, or on its own. This PR makes the first step in this direction by ensuring that the actual Netty service is as small as possible and extracted from Spark core. Given this, we should be able to construct this standalone jar which can be included in other JVMs without incurring significant dependency or runtime issues. The actual work to ensure that such a standalone shuffle service would work in Spark will be left for a future PR, however.

In order to minimize dependencies and allow for the service to be long-running (possibly much longer-running than Spark, and possibly having to support multiple version of Spark simultaneously), the entire service has been ported to Java, where we have full control over the binary compatibility of the components and do not depend on the Scala runtime or version.

These issues: have been addressed by folding in #2330:

SPARK-3453: Refactor Netty module to use BlockTransferService interface
SPARK-3018: Release all buffers upon task completion/failure
SPARK-3002: Create a connection pool and reuse clients across different threads
SPARK-3017: Integration tests and unit tests for connection failures
SPARK-3049: Make sure client doesn't block when server/connection has error(s)
SPARK-3502: SO_RCVBUF and SO_SNDBUF should be bootstrap childOption, not option
SPARK-3503: Disable thread local cache in PooledByteBufAllocator

TODO before mergeable:
- [x] Implement uploadBlock()
- [x] Unit tests for RPC side of code
- [x] Performance testing (see comments [here](https://github.com/apache/spark/pull/2753#issuecomment-59475022))
- [x] Turn OFF by default (currently on for unit testing)

Author: Reynold Xin <rxin@apache.org>
Author: Aaron Davidson <aaron@databricks.com>
Author: cocoatomo <cocoatomo77@gmail.com>
Author: Patrick Wendell <pwendell@gmail.com>
Author: Prashant Sharma <prashant.s@imaginea.com>
Author: Davies Liu <davies.liu@gmail.com>
Author: Anand Avati <avati@redhat.com>

Closes #2753 from aarondav/netty and squashes the following commits:

cadfd28 [Aaron Davidson] Turn netty off by default
d7be11b [Aaron Davidson] Turn netty on by default
4a204b8 [Aaron Davidson] Fail block fetches if client connection fails
2b0d1c0 [Aaron Davidson] 100ch
0c5bca2 [Aaron Davidson] Merge branch 'master' of https://github.com/apache/spark into netty
14e37f7 [Aaron Davidson] Address Reynold's comments
8dfcceb [Aaron Davidson] Merge branch 'master' of https://github.com/apache/spark into netty
322dfc1 [Aaron Davidson] Address Reynold's comments, including major rename
e5675a4 [Aaron Davidson] Fail outstanding RPCs as well
ccd4959 [Aaron Davidson] Don't throw exception if client immediately fails
9da0bc1 [Aaron Davidson] Add RPC unit tests
d236dfd [Aaron Davidson] Remove no-op serializer :)
7b7a26c [Aaron Davidson] Fix Nio compile issue
dd420fd [Aaron Davidson] Merge branch 'master' of https://github.com/apache/spark into netty-test
939f276 [Aaron Davidson] Attempt to make comm. bidirectional
aa58f67 [cocoatomo] [SPARK-3909][PySpark][Doc] A corrupted format in Sphinx documents and building warnings
8dc1ded [cocoatomo] [SPARK-3867][PySpark] ./python/run-tests failed when it run with Python 2.6 and unittest2 is not installed
5b5dbe6 [Prashant Sharma] [SPARK-2924] Required by scala 2.11, only one fun/ctor amongst overriden alternatives, can have default argument(s).
2c5d9dc [Patrick Wendell] HOTFIX: Fix build issue with Akka 2.3.4 upgrade.
020691e [Davies Liu] [SPARK-3886] [PySpark] use AutoBatchedSerializer by default
ae4083a [Anand Avati] [SPARK-2805] Upgrade Akka to 2.3.4
29c6dcf [Aaron Davidson] [SPARK-3453] Netty-based BlockTransferService, extracted from Spark core
f7e7568 [Reynold Xin] Fixed spark.shuffle.io.receiveBuffer setting.
5d98ce3 [Reynold Xin] Flip buffer.
f6c220d [Reynold Xin] Merge with latest master.
407e59a [Reynold Xin] Fix style violation.
a0518c7 [Reynold Xin] Implemented block uploads.
4b18db2 [Reynold Xin] Copy the buffer in fetchBlockSync.
bec4ea2 [Reynold Xin] Removed OIO and added num threads settings.
1bdd7ee [Reynold Xin] Fixed tests.
d68f328 [Reynold Xin] Logging close() in case close() fails.
f63fb4c [Reynold Xin] Add more debug message.
6afc435 [Reynold Xin] Added logging.
c066309 [Reynold Xin] Implement java.io.Closeable interface.
519d64d [Reynold Xin] Mark private package visibility and MimaExcludes.
f0a16e9 [Reynold Xin] Fixed test hanging.
14323a5 [Reynold Xin] Removed BlockManager.getLocalShuffleFromDisk.
b2f3281 [Reynold Xin] Added connection pooling.
d23ed7b [Reynold Xin] Incorporated feedback from Norman: - use same pool for boss and worker - remove ioratio - disable caching of byte buf allocator - childoption sendbuf/receivebuf - fire exception through pipeline
9e0cb87 [Reynold Xin] Fixed BlockClientHandlerSuite
5cd33d7 [Reynold Xin] Fixed style violation.
cb589ec [Reynold Xin] Added more test cases covering cleanup when fault happens in ShuffleBlockFetcherIteratorSuite
1be4e8e [Reynold Xin] Shorten NioManagedBuffer and NettyManagedBuffer class names.
108c9ed [Reynold Xin] Forgot to add TestSerializer to the commit list.
b5c8d1f [Reynold Xin] Fixed ShuffleBlockFetcherIteratorSuite.
064747b [Reynold Xin] Reference count buffers and clean them up properly.
2b44cf1 [Reynold Xin] Added more documentation.
1760d32 [Reynold Xin] Use Epoll.isAvailable in BlockServer as well.
165eab1 [Reynold Xin] [SPARK-3453] Refactor Netty module to use BlockTransferService.
2014-10-29 11:27:07 -07:00
Xiangrui Meng 1559495dd9 [FIX] disable benchmark code
I forgot to disable the benchmark code in #2937, which increased the Jenkins build time by couple minutes.

aarondav

Author: Xiangrui Meng <meng@databricks.com>

Closes #2990 from mengxr/disable-benchmark and squashes the following commits:

c58f070 [Xiangrui Meng] disable benchmark code
2014-10-28 22:00:10 -07:00
zsxwing 1536d70331 [SPARK-4008] Fix "kryo with fold" in KryoSerializerSuite
`zeroValue` will be serialized by `spark.closure.serializer` but `spark.closure.serializer` only supports the default Java serializer. So it must not be `ClassWithoutNoArgConstructor`, which can not be serialized by the Java serializer.

This PR changed `zeroValue` to null and updated the test to make it work correctly.

Author: zsxwing <zsxwing@gmail.com>

Closes #2856 from zsxwing/SPARK-4008 and squashes the following commits:

51da655 [zsxwing] [SPARK-4008] Fix "kryo with fold" in KryoSerializerSuite
2014-10-28 17:59:10 -07:00
Xiangrui Meng 84e5da87e3 [SPARK-4084] Reuse sort key in Sorter
Sorter uses generic-typed key for sorting. When data is large, it creates lots of key objects, which is not efficient. We should reuse the key in Sorter for memory efficiency. This change is part of the petabyte sort implementation from rxin .

The `Sorter` class was written in Java and marked package private. So it is only available to `org.apache.spark.util.collection`. I renamed it to `TimSort` and add a simple wrapper of it, still called `Sorter`, in Scala, which is `private[spark]`.

The benchmark code is updated, which now resets the array before each run. Here is the result on sorting primitive Int arrays of size 25 million using Sorter:

~~~
[info] - Sorter benchmark for key-value pairs !!! IGNORED !!!
Java Arrays.sort() on non-primitive int array: Took 13237 ms
Java Arrays.sort() on non-primitive int array: Took 13320 ms
Java Arrays.sort() on non-primitive int array: Took 15718 ms
Java Arrays.sort() on non-primitive int array: Took 13283 ms
Java Arrays.sort() on non-primitive int array: Took 13267 ms
Java Arrays.sort() on non-primitive int array: Took 15122 ms
Java Arrays.sort() on non-primitive int array: Took 15495 ms
Java Arrays.sort() on non-primitive int array: Took 14877 ms
Java Arrays.sort() on non-primitive int array: Took 16429 ms
Java Arrays.sort() on non-primitive int array: Took 14250 ms
Java Arrays.sort() on non-primitive int array: (13878 ms first try, 14499 ms average)
Java Arrays.sort() on primitive int array: Took 2683 ms
Java Arrays.sort() on primitive int array: Took 2683 ms
Java Arrays.sort() on primitive int array: Took 2701 ms
Java Arrays.sort() on primitive int array: Took 2746 ms
Java Arrays.sort() on primitive int array: Took 2685 ms
Java Arrays.sort() on primitive int array: Took 2735 ms
Java Arrays.sort() on primitive int array: Took 2669 ms
Java Arrays.sort() on primitive int array: Took 2693 ms
Java Arrays.sort() on primitive int array: Took 2680 ms
Java Arrays.sort() on primitive int array: Took 2642 ms
Java Arrays.sort() on primitive int array: (2948 ms first try, 2691 ms average)
Sorter without key reuse on primitive int array: Took 10732 ms
Sorter without key reuse on primitive int array: Took 12482 ms
Sorter without key reuse on primitive int array: Took 10718 ms
Sorter without key reuse on primitive int array: Took 12650 ms
Sorter without key reuse on primitive int array: Took 10747 ms
Sorter without key reuse on primitive int array: Took 10783 ms
Sorter without key reuse on primitive int array: Took 12721 ms
Sorter without key reuse on primitive int array: Took 10604 ms
Sorter without key reuse on primitive int array: Took 10622 ms
Sorter without key reuse on primitive int array: Took 11843 ms
Sorter without key reuse on primitive int array: (11089 ms first try, 11390 ms average)
Sorter with key reuse on primitive int array: Took 5141 ms
Sorter with key reuse on primitive int array: Took 5298 ms
Sorter with key reuse on primitive int array: Took 5066 ms
Sorter with key reuse on primitive int array: Took 5164 ms
Sorter with key reuse on primitive int array: Took 5203 ms
Sorter with key reuse on primitive int array: Took 5274 ms
Sorter with key reuse on primitive int array: Took 5186 ms
Sorter with key reuse on primitive int array: Took 5159 ms
Sorter with key reuse on primitive int array: Took 5164 ms
Sorter with key reuse on primitive int array: Took 5078 ms
Sorter with key reuse on primitive int array: (5311 ms first try, 5173 ms average)
~~~

So with key reuse, it is faster and less likely to trigger GC.

Author: Xiangrui Meng <meng@databricks.com>
Author: Reynold Xin <rxin@apache.org>

Closes #2937 from mengxr/SPARK-4084 and squashes the following commits:

d73c3d0 [Xiangrui Meng] address comments
0b7b682 [Xiangrui Meng] fix mima
a72f53c [Xiangrui Meng] update timeIt
38ba50c [Xiangrui Meng] update timeIt
720f731 [Xiangrui Meng] add doc about JIT specialization
78f2879 [Xiangrui Meng] update tests
7de2efd [Xiangrui Meng] update the Sorter benchmark code to be correct
8626356 [Xiangrui Meng] add prepare to timeIt and update testsin SorterSuite
5f0d530 [Xiangrui Meng] update method modifiers of SortDataFormat
6ffbe66 [Xiangrui Meng] rename Sorter to TimSort and add a Scala wrapper that is private[spark]
b00db4d [Xiangrui Meng] doc and tests
cf94e8a [Xiangrui Meng] renaming
464ddce [Reynold Xin] cherry-pick rxin's commit
2014-10-28 15:14:41 -07:00