git commit: [SPARK-4029][Streaming] Update streaming driver to reliably save and recover received block metadata on driver failures

2014-11-05 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/branch-1.2 f225b3cc1 - 46654b066


[SPARK-4029][Streaming] Update streaming driver to reliably save and recover 
received block metadata on driver failures

As part of the initiative of preventing data loss on driver failure, this JIRA 
tracks the sub task of modifying the streaming driver to reliably save received 
block metadata, and recover them on driver restart.

This was solved by introducing a `ReceivedBlockTracker` that takes all the 
responsibility of managing the metadata of received blocks (i.e. 
`ReceivedBlockInfo`, and any actions on them (e.g, allocating blocks to 
batches, etc.). All actions to block info get written out to a write ahead log 
(using `WriteAheadLogManager`). On recovery, all the actions are replaying to 
recreate the pre-failure state of the `ReceivedBlockTracker`, which include the 
batch-to-block allocations and the unallocated blocks.

Furthermore, the `ReceiverInputDStream` was modified to create 
`WriteAheadLogBackedBlockRDD`s when file segment info is present in the 
`ReceivedBlockInfo`. After recovery of all the block info (through recovery 
`ReceivedBlockTracker`), the `WriteAheadLogBackedBlockRDD`s gets recreated with 
the recovered info, and jobs submitted. The data of the blocks gets pulled from 
the write ahead logs, thanks to the segment info present in the 
`ReceivedBlockInfo`.

This is still a WIP. Things that are missing here are.

- *End-to-end integration tests:* Unit tests that tests the driver recovery, by 
killing and restarting the streaming context, and verifying all the input data 
gets processed. This has been implemented but not included in this PR yet. A 
sneak peek of that DriverFailureSuite can be found in this PR (on my personal 
repo): https://github.com/tdas/spark/pull/25 I can either include it in this 
PR, or submit that as a separate PR after this gets in.

- *WAL cleanup:* Cleaning up the received data write ahead log, by calling 
`ReceivedBlockHandler.cleanupOldBlocks`. This is being worked on.

Author: Tathagata Das tathagata.das1...@gmail.com

Closes #3026 from tdas/driver-ha-rbt and squashes the following commits:

a8009ed [Tathagata Das] Added comment
1d704bb [Tathagata Das] Enabled storing recovered WAL-backed blocks to BM
2ee2484 [Tathagata Das] More minor changes based on PR
47fc1e3 [Tathagata Das] Addressed PR comments.
9a7e3e4 [Tathagata Das] Refactored ReceivedBlockTracker API a bit to make 
things a little cleaner for users of the tracker.
af63655 [Tathagata Das] Minor changes.
fce2b21 [Tathagata Das] Removed commented lines
59496d3 [Tathagata Das] Changed class names, made allocation more explicit and 
added cleanup
19aec7d [Tathagata Das] Fixed casting bug.
f66d277 [Tathagata Das] Fix line lengths.
cda62ee [Tathagata Das] Added license
25611d6 [Tathagata Das] Minor changes before submitting PR
7ae0a7fb [Tathagata Das] Transferred changes from driver-ha-working branch

(cherry picked from commit 5f13759d3642ea5b58c12a756e7125ac19aff10e)
Signed-off-by: Tathagata Das tathagata.das1...@gmail.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/46654b06
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/46654b06
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/46654b06

Branch: refs/heads/branch-1.2
Commit: 46654b0661257f432932c6efc09c4c0983521834
Parents: f225b3c
Author: Tathagata Das tathagata.das1...@gmail.com
Authored: Wed Nov 5 01:21:53 2014 -0800
Committer: Tathagata Das tathagata.das1...@gmail.com
Committed: Wed Nov 5 01:22:16 2014 -0800

--
 .../dstream/ReceiverInputDStream.scala  |  69 +++---
 .../rdd/WriteAheadLogBackedBlockRDD.scala   |   3 +-
 .../streaming/scheduler/JobGenerator.scala  |  21 +-
 .../scheduler/ReceivedBlockTracker.scala| 230 ++
 .../streaming/scheduler/ReceiverTracker.scala   |  98 +---
 .../spark/streaming/BasicOperationsSuite.scala  |  19 +-
 .../streaming/ReceivedBlockTrackerSuite.scala   | 242 +++
 .../rdd/WriteAheadLogBackedBlockRDDSuite.scala  |   4 +-
 8 files changed, 597 insertions(+), 89 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/46654b06/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala
--
diff --git 
a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala
 
b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala
index bb47d37..3e67161 100644
--- 
a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala
+++ 
b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala
@@ -17,15 +17,14 @@
 
 package 

git commit: [SPARK-4197] [mllib] GradientBoosting API cleanup and examples in Scala, Java

2014-11-05 Thread meng
Repository: spark
Updated Branches:
  refs/heads/branch-1.2 46654b066 - 9cba88c7f


[SPARK-4197] [mllib] GradientBoosting API cleanup and examples in Scala, Java

### Summary

* Made it easier to construct default Strategy and BoostingStrategy and to set 
parameters using simple types.
* Added Scala and Java examples for GradientBoostedTrees
* small cleanups and fixes

### Details

GradientBoosting bug fixes (“bug” = bad default options)
* Force boostingStrategy.weakLearnerParams.algo = Regression
* Force boostingStrategy.weakLearnerParams.impurity = impurity.Variance
* Only persist data if not yet persisted (since it causes an error if persisted 
twice)

BoostingStrategy
* numEstimators: renamed to numIterations
* removed subsamplingRate (duplicated by Strategy)
* removed categoricalFeaturesInfo since it belongs with the weak learner params 
(since boosting can be oblivious to feature type)
* Changed algo to var (not val) and added BeanProperty, with overload taking 
String argument
* Added assertValid() method
* Updated defaultParams() method and eliminated defaultWeakLearnerParams() 
since that belongs in Strategy

Strategy (for DecisionTree)
* Changed algo to var (not val) and added BeanProperty, with overload taking 
String argument
* Added setCategoricalFeaturesInfo method taking Java Map.
* Cleaned up assertValid
* Changed val’s to def’s since parameters can now be changed.

CC: manishamde mengxr codedeft

Author: Joseph K. Bradley jos...@databricks.com

Closes #3094 from jkbradley/gbt-api and squashes the following commits:

7a27e22 [Joseph K. Bradley] scalastyle fix
52013d5 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into 
gbt-api
e9b8410 [Joseph K. Bradley] Summary of changes

(cherry picked from commit 5b3b6f6f5f029164d7749366506e142b104c1d43)
Signed-off-by: Xiangrui Meng m...@databricks.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9cba88c7
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9cba88c7
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9cba88c7

Branch: refs/heads/branch-1.2
Commit: 9cba88c7f9fdf151217716e4cc5fa75995736922
Parents: 46654b0
Author: Joseph K. Bradley jos...@databricks.com
Authored: Wed Nov 5 10:33:13 2014 -0800
Committer: Xiangrui Meng m...@databricks.com
Committed: Wed Nov 5 10:33:22 2014 -0800

--
 .../mllib/JavaGradientBoostedTrees.java | 126 ++
 .../examples/mllib/DecisionTreeRunner.scala |  64 ---
 .../examples/mllib/GradientBoostedTrees.scala   | 146 
 .../spark/mllib/tree/GradientBoosting.scala | 169 ++-
 .../tree/configuration/BoostingStrategy.scala   |  78 -
 .../mllib/tree/configuration/Strategy.scala |  51 --
 .../mllib/tree/GradientBoostingSuite.scala  |  34 ++--
 7 files changed, 462 insertions(+), 206 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/9cba88c7/examples/src/main/java/org/apache/spark/examples/mllib/JavaGradientBoostedTrees.java
--
diff --git 
a/examples/src/main/java/org/apache/spark/examples/mllib/JavaGradientBoostedTrees.java
 
b/examples/src/main/java/org/apache/spark/examples/mllib/JavaGradientBoostedTrees.java
new file mode 100644
index 000..1af2067
--- /dev/null
+++ 
b/examples/src/main/java/org/apache/spark/examples/mllib/JavaGradientBoostedTrees.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.examples.mllib;
+
+import scala.Tuple2;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.api.java.function.Function2;
+import org.apache.spark.api.java.function.PairFunction;
+import org.apache.spark.mllib.regression.LabeledPoint;
+import org.apache.spark.mllib.tree.GradientBoosting;
+import 

[3/3] git commit: [SPARK-4242] [Core] Add SASL to external shuffle service

2014-11-05 Thread pwendell
[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 aa...@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


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e7f73563
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e7f73563
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e7f73563

Branch: refs/heads/branch-1.2
Commit: e7f735637ad2f681b454d1297f6fdcc433feebbc
Parents: 2364340
Author: Aaron Davidson aa...@databricks.com
Authored: Wed Nov 5 14:38:43 2014 -0800
Committer: Patrick Wendell pwend...@gmail.com
Committed: Wed Nov 5 14:41:13 2014 -0800

--
 LICENSE |  21 +++-
 .../main/scala/org/apache/spark/SparkEnv.scala  |   2 +-
 .../org/apache/spark/storage/BlockManager.scala |  12 +-
 .../storage/BlockManagerReplicationSuite.scala  |   4 +-
 .../spark/storage/BlockManagerSuite.scala   |   4 +-
 network/common/pom.xml  |   1 +
 .../buffer/FileSegmentManagedBuffer.java|   3 +-
 .../spark/network/util/LimitedInputStream.java  |  87 ++
 network/shuffle/pom.xml |   1 +
 .../spark/network/sasl/SparkSaslClient.java |   1 -
 .../spark/network/sasl/SparkSaslServer.java |   9 +-
 .../network/shuffle/ExternalShuffleClient.java  |  31 -
 .../ExternalShuffleIntegrationSuite.java|   4 +-
 .../shuffle/ExternalShuffleSecuritySuite.java   | 113 +++
 .../streaming/ReceivedBlockHandlerSuite.scala   |   2 +-
 15 files changed, 272 insertions(+), 23 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e7f73563/LICENSE
--
diff --git a/LICENSE b/LICENSE
index f1732fb..3c667bf 100644
--- a/LICENSE
+++ b/LICENSE
@@ -754,7 +754,7 @@ SUCH DAMAGE.
 
 
 
-For Timsort (core/src/main/java/org/apache/spark/util/collection/Sorter.java):
+For Timsort (core/src/main/java/org/apache/spark/util/collection/TimSort.java):
 
 Copyright (C) 2008 The Android Open Source Project
 
@@ -772,6 +772,25 @@ limitations under the License.
 
 
 
+For LimitedInputStream
+  
(network/common/src/main/java/org/apache/spark/network/util/LimitedInputStream.java):
+
+Copyright (C) 2007 The Guava Authors
+
+Licensed under the Apache License, Version 2.0 (the License);
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an AS IS BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+
+
 BSD-style licenses
 
 

http://git-wip-us.apache.org/repos/asf/spark/blob/e7f73563/core/src/main/scala/org/apache/spark/SparkEnv.scala
--
diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala 
b/core/src/main/scala/org/apache/spark/SparkEnv.scala
index 45e9d7f..e7454be 100644
--- a/core/src/main/scala/org/apache/spark/SparkEnv.scala
+++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala
@@ -287,7 +287,7 @@ object SparkEnv extends Logging {
 
 // NB: blockManager is not valid until initialize() is called later.
 val blockManager = new BlockManager(executorId, actorSystem, 
blockManagerMaster,
-  serializer, conf, mapOutputTracker, shuffleManager, blockTransferService)
+  serializer, conf, mapOutputTracker, shuffleManager, 
blockTransferService, securityManager)
 
 val broadcastManager = new BroadcastManager(isDriver, conf, 
securityManager)
 


[2/3] git commit: [SPARK-2938] Support SASL authentication in NettyBlockTransferService

2014-11-05 Thread pwendell
[SPARK-2938] Support SASL authentication in NettyBlockTransferService

Also lays the groundwork for supporting it inside the external shuffle service.

Author: Aaron Davidson aa...@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


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/23643403
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/23643403
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/23643403

Branch: refs/heads/branch-1.2
Commit: 236434033fe452e70dbd0236935a49693712e130
Parents: 9cba88c
Author: Aaron Davidson aa...@databricks.com
Authored: Tue Nov 4 16:15:38 2014 -0800
Committer: Patrick Wendell pwend...@gmail.com
Committed: Wed Nov 5 14:41:03 2014 -0800

--
 .../org/apache/spark/SecurityManager.scala  |  23 ++-
 .../main/scala/org/apache/spark/SparkConf.scala |   6 +
 .../scala/org/apache/spark/SparkContext.scala   |   2 +
 .../main/scala/org/apache/spark/SparkEnv.scala  |   3 +-
 .../org/apache/spark/SparkSaslClient.scala  | 147 
 .../org/apache/spark/SparkSaslServer.scala  | 176 ---
 .../org/apache/spark/executor/Executor.scala|   1 +
 .../netty/NettyBlockTransferService.scala   |  28 ++-
 .../apache/spark/network/nio/Connection.scala   |   5 +-
 .../spark/network/nio/ConnectionManager.scala   |   7 +-
 .../org/apache/spark/storage/BlockManager.scala |  45 +++--
 .../netty/NettyBlockTransferSecuritySuite.scala | 161 +
 .../network/nio/ConnectionManagerSuite.scala|   6 +-
 .../storage/BlockManagerReplicationSuite.scala  |   2 +
 .../spark/storage/BlockManagerSuite.scala   |   4 +-
 docs/security.md|   1 -
 .../apache/spark/network/TransportContext.java  |  15 +-
 .../spark/network/client/TransportClient.java   |  11 +-
 .../client/TransportClientBootstrap.java|  32 
 .../network/client/TransportClientFactory.java  |  64 +--
 .../spark/network/server/NoOpRpcHandler.java|   2 +-
 .../apache/spark/network/server/RpcHandler.java |  19 +-
 .../network/server/TransportRequestHandler.java |   1 +
 .../spark/network/util/TransportConf.java   |   3 +
 .../spark/network/sasl/SaslClientBootstrap.java |  74 
 .../apache/spark/network/sasl/SaslMessage.java  |  74 
 .../spark/network/sasl/SaslRpcHandler.java  |  97 ++
 .../spark/network/sasl/SecretKeyHolder.java |  35 
 .../spark/network/sasl/SparkSaslClient.java | 138 +++
 .../spark/network/sasl/SparkSaslServer.java | 170 ++
 .../shuffle/ExternalShuffleBlockHandler.java|   2 +-
 .../network/shuffle/ExternalShuffleClient.java  |  15 +-
 .../spark/network/shuffle/ShuffleClient.java|  11 +-
 .../network/sasl/SaslIntegrationSuite.java  | 172 ++
 .../spark/network/sasl/SparkSaslSuite.java  |  89 ++
 .../ExternalShuffleIntegrationSuite.java|   7 +-
 .../streaming/ReceivedBlockHandlerSuite.scala   |   1 +
 37 files changed, 1257 insertions(+), 392 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/23643403/core/src/main/scala/org/apache/spark/SecurityManager.scala
--
diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala 
b/core/src/main/scala/org/apache/spark/SecurityManager.scala
index 0e0f1a7..dee935f 100644
--- a/core/src/main/scala/org/apache/spark/SecurityManager.scala
+++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala
@@ -22,6 +22,7 @@ import java.net.{Authenticator, PasswordAuthentication}
 import org.apache.hadoop.io.Text
 
 import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.network.sasl.SecretKeyHolder
 
 /**
  * Spark class responsible for security.
@@ -84,7 +85,7 @@ import org.apache.spark.deploy.SparkHadoopUtil
  *Authenticator installed in the SecurityManager to how it does 
the authentication
  *and in this case gets the user name and password from the 
request.
  *
- *  - ConnectionManager - The Spark ConnectionManager uses java nio to 
asynchronously
+ *  - BlockTransferService - The Spark 

svn commit: r1636996 [1/2] - in /spark: ./ _layouts/ site/ site/graphx/ site/mllib/ site/news/ site/releases/ site/screencasts/ site/sql/ site/streaming/

2014-11-05 Thread rxin
Author: rxin
Date: Wed Nov  5 22:42:47 2014
New Revision: 1636996

URL: http://svn.apache.org/r1636996
Log:
Added sort benchmark news and various minor updates.

Modified:
spark/_layouts/global.html
spark/community.md
spark/faq.md
spark/site/community.html
spark/site/documentation.html
spark/site/downloads.html
spark/site/examples.html
spark/site/faq.html
spark/site/graphx/index.html
spark/site/index.html
spark/site/mailing-lists.html
spark/site/mllib/index.html
spark/site/news/amp-camp-2013-registration-ope.html
spark/site/news/announcing-the-first-spark-summit.html
spark/site/news/fourth-spark-screencast-published.html
spark/site/news/index.html
spark/site/news/nsdi-paper.html
spark/site/news/proposals-open-for-spark-summit-east.html
spark/site/news/run-spark-and-shark-on-amazon-emr.html
spark/site/news/spark-0-6-1-and-0-5-2-released.html
spark/site/news/spark-0-6-2-released.html
spark/site/news/spark-0-7-0-released.html
spark/site/news/spark-0-7-2-released.html
spark/site/news/spark-0-7-3-released.html
spark/site/news/spark-0-8-0-released.html
spark/site/news/spark-0-8-1-released.html
spark/site/news/spark-0-9-0-released.html
spark/site/news/spark-0-9-1-released.html
spark/site/news/spark-0-9-2-released.html
spark/site/news/spark-1-0-0-released.html
spark/site/news/spark-1-0-1-released.html
spark/site/news/spark-1-0-2-released.html
spark/site/news/spark-1-1-0-released.html
spark/site/news/spark-accepted-into-apache-incubator.html
spark/site/news/spark-and-shark-in-the-news.html
spark/site/news/spark-becomes-tlp.html
spark/site/news/spark-featured-in-wired.html
spark/site/news/spark-mailing-lists-moving-to-apache.html
spark/site/news/spark-meetups.html
spark/site/news/spark-screencasts-published.html
spark/site/news/spark-summit-2013-is-a-wrap.html
spark/site/news/spark-summit-2014-videos-posted.html
spark/site/news/spark-summit-agenda-posted.html
spark/site/news/spark-tips-from-quantifind.html
spark/site/news/spark-user-survey-and-powered-by-page.html
spark/site/news/spark-version-0-6-0-released.html
spark/site/news/strata-exercises-now-available-online.html
spark/site/news/submit-talks-to-spark-summit-2014.html
spark/site/news/two-weeks-to-spark-summit-2014.html
spark/site/news/video-from-first-spark-development-meetup.html
spark/site/releases/spark-release-0-3.html
spark/site/releases/spark-release-0-5-0.html
spark/site/releases/spark-release-0-5-1.html
spark/site/releases/spark-release-0-5-2.html
spark/site/releases/spark-release-0-6-0.html
spark/site/releases/spark-release-0-6-1.html
spark/site/releases/spark-release-0-6-2.html
spark/site/releases/spark-release-0-7-0.html
spark/site/releases/spark-release-0-7-2.html
spark/site/releases/spark-release-0-7-3.html
spark/site/releases/spark-release-0-8-0.html
spark/site/releases/spark-release-0-8-1.html
spark/site/releases/spark-release-0-9-0.html
spark/site/releases/spark-release-0-9-1.html
spark/site/releases/spark-release-0-9-2.html
spark/site/releases/spark-release-1-0-0.html
spark/site/releases/spark-release-1-0-1.html
spark/site/releases/spark-release-1-0-2.html
spark/site/releases/spark-release-1-1-0.html
spark/site/research.html
spark/site/screencasts/1-first-steps-with-spark.html
spark/site/screencasts/2-spark-documentation-overview.html
spark/site/screencasts/3-transformations-and-caching.html
spark/site/screencasts/4-a-standalone-job-in-spark.html
spark/site/screencasts/index.html
spark/site/sql/index.html
spark/site/streaming/index.html

Modified: spark/_layouts/global.html
URL: 
http://svn.apache.org/viewvc/spark/_layouts/global.html?rev=1636996r1=1636995r2=1636996view=diff
==
--- spark/_layouts/global.html (original)
+++ spark/_layouts/global.html Wed Nov  5 22:42:47 2014
@@ -117,7 +117,6 @@
 ul class=dropdown-menu
   lia href={{site.url}}documentation.htmlOverview/a/li
   lia href={{site.url}}docs/latest/Latest Release (Spark 
1.1.0)/a/li
-  lia href={{site.url}}examples.htmlExamples/a/li
 /ul
   /li
   li class=dropdown
@@ -131,6 +130,7 @@
   lia 
href=https://cwiki.apache.org/confluence/display/SPARK/Powered+By+Spark;Powered
 By/a/li
 /ul
   /li
+  lia href={{site.url}}examples.htmlExamples/a/li
   lia href={{site.url}}faq.htmlFAQ/a/li
 /ul
   /div

Modified: spark/community.md
URL: 
http://svn.apache.org/viewvc/spark/community.md?rev=1636996r1=1636995r2=1636996view=diff
==
--- spark/community.md (original)
+++ spark/community.md Wed Nov  5 22:42:47 2014
@@ -109,12 +109,12 @@ in 2009, and was open sourced 

svn commit: r1636996 [2/2] - in /spark: ./ _layouts/ site/ site/graphx/ site/mllib/ site/news/ site/releases/ site/screencasts/ site/sql/ site/streaming/

2014-11-05 Thread rxin
Modified: spark/site/news/spark-tips-from-quantifind.html
URL: 
http://svn.apache.org/viewvc/spark/site/news/spark-tips-from-quantifind.html?rev=1636996r1=1636995r2=1636996view=diff
==
--- spark/site/news/spark-tips-from-quantifind.html (original)
+++ spark/site/news/spark-tips-from-quantifind.html Wed Nov  5 22:42:47 2014
@@ -102,7 +102,6 @@
 ul class=dropdown-menu
   lia href=/documentation.htmlOverview/a/li
   lia href=/docs/latest/Latest Release (Spark 1.1.0)/a/li
-  lia href=/examples.htmlExamples/a/li
 /ul
   /li
   li class=dropdown
@@ -116,6 +115,7 @@
   lia 
href=https://cwiki.apache.org/confluence/display/SPARK/Powered+By+Spark;Powered
 By/a/li
 /ul
   /li
+  lia href=/examples.htmlExamples/a/li
   lia href=/faq.htmlFAQ/a/li
 /ul
   /div
@@ -129,6 +129,9 @@
   h5Latest News/h5
   ul class=list-unstyled
 
+  lia 
href=/news/spark-wins-daytona-gray-sort-100tb-benchmark.htmlSpark wins 
Daytona Gray Sort 100TB Benchmark/a
+  span class=small(Nov 05, 2014)/span/li
+
   lia 
href=/news/proposals-open-for-spark-summit-east.htmlSubmissions open for 
Spark Summit East 2015 in New York/a
   span class=small(Oct 18, 2014)/span/li
 
@@ -138,9 +141,6 @@
   lia href=/news/spark-1-0-2-released.htmlSpark 1.0.2 
released/a
   span class=small(Aug 05, 2014)/span/li
 
-  lia href=/news/spark-0-9-2-released.htmlSpark 0.9.2 
released/a
-  span class=small(Jul 23, 2014)/span/li
-
   /ul
   p class=small style=text-align: right;a 
href=/news/index.htmlArchive/a/p
 /div

Modified: spark/site/news/spark-user-survey-and-powered-by-page.html
URL: 
http://svn.apache.org/viewvc/spark/site/news/spark-user-survey-and-powered-by-page.html?rev=1636996r1=1636995r2=1636996view=diff
==
--- spark/site/news/spark-user-survey-and-powered-by-page.html (original)
+++ spark/site/news/spark-user-survey-and-powered-by-page.html Wed Nov  5 
22:42:47 2014
@@ -102,7 +102,6 @@
 ul class=dropdown-menu
   lia href=/documentation.htmlOverview/a/li
   lia href=/docs/latest/Latest Release (Spark 1.1.0)/a/li
-  lia href=/examples.htmlExamples/a/li
 /ul
   /li
   li class=dropdown
@@ -116,6 +115,7 @@
   lia 
href=https://cwiki.apache.org/confluence/display/SPARK/Powered+By+Spark;Powered
 By/a/li
 /ul
   /li
+  lia href=/examples.htmlExamples/a/li
   lia href=/faq.htmlFAQ/a/li
 /ul
   /div
@@ -129,6 +129,9 @@
   h5Latest News/h5
   ul class=list-unstyled
 
+  lia 
href=/news/spark-wins-daytona-gray-sort-100tb-benchmark.htmlSpark wins 
Daytona Gray Sort 100TB Benchmark/a
+  span class=small(Nov 05, 2014)/span/li
+
   lia 
href=/news/proposals-open-for-spark-summit-east.htmlSubmissions open for 
Spark Summit East 2015 in New York/a
   span class=small(Oct 18, 2014)/span/li
 
@@ -138,9 +141,6 @@
   lia href=/news/spark-1-0-2-released.htmlSpark 1.0.2 
released/a
   span class=small(Aug 05, 2014)/span/li
 
-  lia href=/news/spark-0-9-2-released.htmlSpark 0.9.2 
released/a
-  span class=small(Jul 23, 2014)/span/li
-
   /ul
   p class=small style=text-align: right;a 
href=/news/index.htmlArchive/a/p
 /div

Modified: spark/site/news/spark-version-0-6-0-released.html
URL: 
http://svn.apache.org/viewvc/spark/site/news/spark-version-0-6-0-released.html?rev=1636996r1=1636995r2=1636996view=diff
==
--- spark/site/news/spark-version-0-6-0-released.html (original)
+++ spark/site/news/spark-version-0-6-0-released.html Wed Nov  5 22:42:47 2014
@@ -102,7 +102,6 @@
 ul class=dropdown-menu
   lia href=/documentation.htmlOverview/a/li
   lia href=/docs/latest/Latest Release (Spark 1.1.0)/a/li
-  lia href=/examples.htmlExamples/a/li
 /ul
   /li
   li class=dropdown
@@ -116,6 +115,7 @@
   lia 
href=https://cwiki.apache.org/confluence/display/SPARK/Powered+By+Spark;Powered
 By/a/li
 /ul
   /li
+  lia href=/examples.htmlExamples/a/li
   lia href=/faq.htmlFAQ/a/li
 /ul
   /div
@@ -129,6 +129,9 @@
   h5Latest News/h5
   ul class=list-unstyled
 
+  lia 
href=/news/spark-wins-daytona-gray-sort-100tb-benchmark.htmlSpark wins 
Daytona Gray Sort 100TB Benchmark/a
+  span class=small(Nov 05, 2014)/span/li
+
   lia 
href=/news/proposals-open-for-spark-summit-east.htmlSubmissions open for 
Spark Summit East 2015 in New York/a
   span class=small(Oct 18, 2014)/span/li
 
@@ -138,9 +141,6 @@
   lia 

[1/3] [SPARK-2938] Support SASL authentication in NettyBlockTransferService

2014-11-05 Thread pwendell
Repository: spark
Updated Branches:
  refs/heads/branch-1.2 9cba88c7f - e7f735637


http://git-wip-us.apache.org/repos/asf/spark/blob/23643403/network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java
--
diff --git 
a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java
 
b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java
new file mode 100644
index 000..2c0ce40
--- /dev/null
+++ 
b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java
@@ -0,0 +1,170 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.sasl;
+
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.RealmCallback;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import java.io.IOException;
+import java.util.Map;
+
+import com.google.common.base.Charsets;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.io.BaseEncoding;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A SASL Server for Spark which simply keeps track of the state of a single 
SASL session, from the
+ * initial state to the authenticated state. (It is not a server in the 
sense of accepting
+ * connections on some socket.)
+ */
+public class SparkSaslServer {
+  private final Logger logger = LoggerFactory.getLogger(SparkSaslServer.class);
+
+  /**
+   * This is passed as the server name when creating the sasl client/server.
+   * This could be changed to be configurable in the future.
+   */
+  static final String DEFAULT_REALM = default;
+
+  /**
+   * The authentication mechanism used here is DIGEST-MD5. This could be 
changed to be
+   * configurable in the future.
+   */
+  static final String DIGEST = DIGEST-MD5;
+
+  /**
+   * The quality of protection is just auth. This means that we are doing
+   * authentication only, we are not supporting integrity or privacy 
protection of the
+   * communication channel after authentication. This could be changed to be 
configurable
+   * in the future.
+   */
+  static final MapString, String SASL_PROPS = ImmutableMap.String, 
Stringbuilder()
+.put(Sasl.QOP, auth)
+.put(Sasl.SERVER_AUTH, true)
+.build();
+
+  /** Identifier for a certain secret key within the secretKeyHolder. */
+  private final String secretKeyId;
+  private final SecretKeyHolder secretKeyHolder;
+  private SaslServer saslServer;
+
+  public SparkSaslServer(String secretKeyId, SecretKeyHolder secretKeyHolder) {
+this.secretKeyId = secretKeyId;
+this.secretKeyHolder = secretKeyHolder;
+try {
+  this.saslServer = Sasl.createSaslServer(DIGEST, null, DEFAULT_REALM, 
SASL_PROPS,
+new DigestCallbackHandler());
+} catch (SaslException e) {
+  throw Throwables.propagate(e);
+}
+  }
+
+  /**
+   * Determines whether the authentication exchange has completed successfully.
+   */
+  public synchronized boolean isComplete() {
+return saslServer != null  saslServer.isComplete();
+  }
+
+  /**
+   * Used to respond to server SASL tokens.
+   * @param token Server's SASL token
+   * @return response to send back to the server.
+   */
+  public synchronized byte[] response(byte[] token) {
+try {
+  return saslServer != null ? saslServer.evaluateResponse(token) : new 
byte[0];
+} catch (SaslException e) {
+  throw Throwables.propagate(e);
+}
+  }
+
+  /**
+   * Disposes of any system resources or security-sensitive information the
+   * SaslServer might be using.
+   */
+  public synchronized void dispose() {
+if (saslServer != null) {
+  try {
+saslServer.dispose();
+  } catch (SaslException e) {
+// ignore
+  } finally {
+ 

git commit: [SPARK-611] Display executor thread dumps in web UI

2014-11-05 Thread pwendell
Repository: spark
Updated Branches:
  refs/heads/branch-1.2 e7f735637 - 866c7bbe5


[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 joshro...@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


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/866c7bbe
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/866c7bbe
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/866c7bbe

Branch: refs/heads/branch-1.2
Commit: 866c7bbe56f9c7fd96d3f4afe8a76405dc877a6e
Parents: e7f7356
Author: Josh Rosen joshro...@databricks.com
Authored: Mon Nov 3 18:18:47 2014 -0800
Committer: Patrick Wendell pwend...@gmail.com
Committed: Wed Nov 5 14:44:24 2014 -0800

--
 .../scala/org/apache/spark/SparkContext.scala   | 29 +++-
 .../executor/CoarseGrainedExecutorBackend.scala |  3 +-
 .../org/apache/spark/executor/Executor.scala|  7 +-
 .../apache/spark/executor/ExecutorActor.scala   | 41 +++
 .../spark/storage/BlockManagerMaster.scala  |  4 ++
 .../spark/storage/BlockManagerMasterActor.scala | 18 +
 .../spark/storage/BlockManagerMessages.scala|  2 +
 .../spark/ui/exec/ExecutorThreadDumpPage.scala  | 73 
 .../apache/spark/ui/exec/ExecutorsPage.scala| 15 +++-
 .../org/apache/spark/ui/exec/ExecutorsTab.scala |  8 ++-
 .../scala/org/apache/spark/util/AkkaUtils.scala | 14 
 .../apache/spark/util/ThreadStackTrace.scala| 27 
 .../scala/org/apache/spark/util/Utils.scala | 13 
 13 files changed, 247 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/866c7bbe/core/src/main/scala/org/apache/spark/SparkContext.scala
--
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala 
b/core/src/main/scala/org/apache/spark/SparkContext.scala
index d65027d..3cdaa6a 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -21,9 +21,8 @@ import scala.language.implicitConversions
 
 import java.io._
 import java.net.URI
-import java.util.Arrays
+import java.util.{Arrays, Properties, UUID}
 import java.util.concurrent.atomic.AtomicInteger
-import java.util.{Properties, UUID}
 import java.util.UUID.randomUUID
 import scala.collection.{Map, Set}
 import scala.collection.generic.Growable
@@ -41,6 +40,7 @@ import akka.actor.Props
 import org.apache.spark.annotation.{DeveloperApi, Experimental}
 import org.apache.spark.broadcast.Broadcast
 import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil}
+import org.apache.spark.executor.TriggerThreadDump
 import 

[3/4] git commit: [SPARK-4166][Core] Add a backward compatibility test for ExecutorLostFailure

2014-11-05 Thread pwendell
[SPARK-4166][Core] Add a backward compatibility test for ExecutorLostFailure

Author: zsxwing zsxw...@gmail.com

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

89329f4 [zsxwing] Add a backward compatibility test for ExecutorLostFailure


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/68be37b8
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/68be37b8
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/68be37b8

Branch: refs/heads/branch-1.2
Commit: 68be37b823516dbeda066776bb060bf894db4e95
Parents: e0a043b
Author: zsxwing zsxw...@gmail.com
Authored: Mon Nov 3 22:47:45 2014 -0800
Committer: Patrick Wendell pwend...@gmail.com
Committed: Wed Nov 5 14:46:54 2014 -0800

--
 .../scala/org/apache/spark/util/JsonProtocolSuite.scala | 9 +
 1 file changed, 9 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/68be37b8/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
--
diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala 
b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
index 0103012..aec1e40 100644
--- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
@@ -196,6 +196,15 @@ class JsonProtocolSuite extends FunSuite {
 assert(applicationStart === 
JsonProtocol.applicationStartFromJson(oldEvent))
   }
 
+  test(ExecutorLostFailure backward compatibility) {
+// ExecutorLostFailure in Spark 1.1.0 does not have an Executor ID 
property.
+val executorLostFailure = ExecutorLostFailure(100)
+val oldEvent = JsonProtocol.taskEndReasonToJson(executorLostFailure)
+  .removeField({ _._1 == Executor ID })
+val expectedExecutorLostFailure = ExecutorLostFailure(Unknown)
+assert(expectedExecutorLostFailure === 
JsonProtocol.taskEndReasonFromJson(oldEvent))
+  }
+
   /** -- *
| Helper test running methods |
* --- */


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[2/4] git commit: [SPARK-4163][Core] Add a backward compatibility test for FetchFailed

2014-11-05 Thread pwendell
[SPARK-4163][Core] Add a backward compatibility test for FetchFailed

/cc aarondav

Author: zsxwing zsxw...@gmail.com

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

21cb2a8 [zsxwing] Add a backward compatibility test for FetchFailed


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e0a043b7
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e0a043b7
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e0a043b7

Branch: refs/heads/branch-1.2
Commit: e0a043b79c250515a680485f0dc7b1a149835445
Parents: 7517c37
Author: zsxwing zsxw...@gmail.com
Authored: Mon Nov 3 22:40:43 2014 -0800
Committer: Patrick Wendell pwend...@gmail.com
Committed: Wed Nov 5 14:46:46 2014 -0800

--
 .../scala/org/apache/spark/util/JsonProtocolSuite.scala  | 11 +++
 1 file changed, 11 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e0a043b7/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
--
diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala 
b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
index a91c9dd..0103012 100644
--- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
@@ -177,6 +177,17 @@ class JsonProtocolSuite extends FunSuite {
   deserializedBmRemoved)
   }
 
+  test(FetchFailed backwards compatibility) {
+// FetchFailed in Spark 1.1.0 does not have an Message property.
+val fetchFailed = FetchFailed(BlockManagerId(With or, without you, 
15), 17, 18, 19,
+  ignored)
+val oldEvent = JsonProtocol.taskEndReasonToJson(fetchFailed)
+  .removeField({ _._1 == Message })
+val expectedFetchFailed = FetchFailed(BlockManagerId(With or, without 
you, 15), 17, 18, 19,
+  Unknown reason)
+assert(expectedFetchFailed === 
JsonProtocol.taskEndReasonFromJson(oldEvent))
+  }
+
   test(SparkListenerApplicationStart backwards compatibility) {
 // SparkListenerApplicationStart in Spark 1.0.0 do not have an appId 
property.
 val applicationStart = SparkListenerApplicationStart(test, None, 1L, 
user)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[4/4] git commit: [EC2] Factor out Mesos spark-ec2 branch

2014-11-05 Thread pwendell
[EC2] Factor out Mesos spark-ec2 branch

We reference a specific branch in two places. This patch makes it one place.

Author: Nicholas Chammas nicholas.cham...@gmail.com

Closes #3008 from nchammas/mesos-spark-ec2-branch and squashes the following 
commits:

10a6089 [Nicholas Chammas] factor out mess spark-ec2 branch


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b27d7dca
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b27d7dca
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b27d7dca

Branch: refs/heads/branch-1.2
Commit: b27d7dcaaad0bf04d341660ffbeb742cd4eecfd3
Parents: 68be37b
Author: Nicholas Chammas nicholas.cham...@gmail.com
Authored: Mon Nov 3 09:02:35 2014 -0800
Committer: Patrick Wendell pwend...@gmail.com
Committed: Wed Nov 5 14:47:29 2014 -0800

--
 ec2/spark_ec2.py | 11 +--
 1 file changed, 9 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/b27d7dca/ec2/spark_ec2.py
--
diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py
index 0d6b82b..50f88f7 100755
--- a/ec2/spark_ec2.py
+++ b/ec2/spark_ec2.py
@@ -41,8 +41,9 @@ from boto import ec2
 
 DEFAULT_SPARK_VERSION = 1.1.0
 
+MESOS_SPARK_EC2_BRANCH = v4
 # A URL prefix from which to fetch AMI information
-AMI_PREFIX = https://raw.github.com/mesos/spark-ec2/v2/ami-list;
+AMI_PREFIX = 
https://raw.github.com/mesos/spark-ec2/{b}/ami-list.format(b=MESOS_SPARK_EC2_BRANCH)
 
 
 class UsageError(Exception):
@@ -583,7 +584,13 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, 
deploy_ssh_key):
 
 # NOTE: We should clone the repository before running deploy_files to
 # prevent ec2-variables.sh from being overwritten
-ssh(master, opts, rm -rf spark-ec2  git clone 
https://github.com/mesos/spark-ec2.git -b v4)
+ssh(
+host=master,
+opts=opts,
+command=rm -rf spark-ec2
++   
++ git clone https://github.com/mesos/spark-ec2.git -b 
{b}.format(b=MESOS_SPARK_EC2_BRANCH)
+)
 
 print Deploying files to master...
 deploy_files(conn, deploy.generic, opts, master_nodes, slave_nodes, 
modules)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



svn commit: r1637002 - in /spark: news/_posts/2014-11-05-spark-wins-daytona-gray-sort-100tb-benchmark.md site/news/spark-wins-daytona-gray-sort-100tb-benchmark.html

2014-11-05 Thread rxin
Author: rxin
Date: Wed Nov  5 23:01:17 2014
New Revision: 1637002

URL: http://svn.apache.org/r1637002
Log:
added the sort benchmark news item

Added:
spark/news/_posts/2014-11-05-spark-wins-daytona-gray-sort-100tb-benchmark.md
spark/site/news/spark-wins-daytona-gray-sort-100tb-benchmark.html

Added: 
spark/news/_posts/2014-11-05-spark-wins-daytona-gray-sort-100tb-benchmark.md
URL: 
http://svn.apache.org/viewvc/spark/news/_posts/2014-11-05-spark-wins-daytona-gray-sort-100tb-benchmark.md?rev=1637002view=auto
==
--- 
spark/news/_posts/2014-11-05-spark-wins-daytona-gray-sort-100tb-benchmark.md 
(added)
+++ 
spark/news/_posts/2014-11-05-spark-wins-daytona-gray-sort-100tb-benchmark.md 
Wed Nov  5 23:01:17 2014
@@ -0,0 +1,21 @@
+---
+layout: post
+title: Spark wins Daytona Gray Sort 100TB Benchmark
+categories:
+- News
+tags: []
+status: publish
+type: post
+published: true
+meta:
+  _edit_last: '4'
+  _wpas_done_all: '1'
+---
+
+We are proud to announce that Spark won the a 
href=http://sortbenchmark.org/;2014 Gray Sort Benchmark/a (Daytona 100TB 
category). A team from a href=http://databricks.com/;Databricks/a 
including Spark committers, Reynold Xin, Xiangrui Meng, and Matei Zaharia, a 
href=http://databricks.com/blog/2014/11/05/spark-officially-sets-a-new-record-in-large-scale-sorting.html;entered
 the benchmark using Spark/a. Spark won a tie with the Themis team from UCSD, 
and jointly set a new world record in sorting.
+
+They used Spark and sorted 100TB of data using 206 EC2 i2.8xlarge machines in 
23 minutes. The previous world record was 72 minutes, set by a Hadoop MapReduce 
cluster of 2100 nodes. This means that Spark sorted the same data 3X faster 
using 10X fewer machines. All the sorting took place on disk (HDFS), without 
using Spark’s in-memory cache.
+
+Outperforming large Hadoop MapReduce clusters on sorting not only validates 
the vision and work done by the Spark community, but also demonstrates that 
Spark is fulfilling its promise to serve as a faster and more scalable engine 
for data processing of all sizes.
+
+For more information, see the a 
href=http://databricks.com/blog/2014/11/05/spark-officially-sets-a-new-record-in-large-scale-sorting.html;Databricks
 blog article/a written by the Reynold Xin.

Added: spark/site/news/spark-wins-daytona-gray-sort-100tb-benchmark.html
URL: 
http://svn.apache.org/viewvc/spark/site/news/spark-wins-daytona-gray-sort-100tb-benchmark.html?rev=1637002view=auto
==
--- spark/site/news/spark-wins-daytona-gray-sort-100tb-benchmark.html (added)
+++ spark/site/news/spark-wins-daytona-gray-sort-100tb-benchmark.html Wed Nov  
5 23:01:17 2014
@@ -0,0 +1,196 @@
+!DOCTYPE html
+html lang=en
+head
+  meta charset=utf-8
+  meta http-equiv=X-UA-Compatible content=IE=edge
+  meta name=viewport content=width=device-width, initial-scale=1.0
+
+  title
+ Spark wins Daytona Gray Sort 100TB Benchmark | Apache Spark
+
+  /title
+
+  
+
+  !-- Bootstrap core CSS --
+  link href=/css/cerulean.min.css rel=stylesheet
+  link href=/css/custom.css rel=stylesheet
+
+  script type=text/javascript
+  !-- Google Analytics initialization --
+  var _gaq = _gaq || [];
+  _gaq.push(['_setAccount', 'UA-32518208-2']);
+  _gaq.push(['_trackPageview']);
+  (function() {
+var ga = document.createElement('script'); ga.type = 'text/javascript'; 
ga.async = true;
+ga.src = ('https:' == document.location.protocol ? 'https://ssl' : 
'http://www') + '.google-analytics.com/ga.js';
+var s = document.getElementsByTagName('script')[0]; 
s.parentNode.insertBefore(ga, s);
+  })();
+
+  !-- Adds slight delay to links to allow async reporting --
+  function trackOutboundLink(link, category, action) {
+try {
+  _gaq.push(['_trackEvent', category , action]);
+} catch(err){}
+
+setTimeout(function() {
+  document.location.href = link.href;
+}, 100);
+  }
+  /script
+
+  !-- HTML5 shim and Respond.js IE8 support of HTML5 elements and media 
queries --
+  !--[if lt IE 9]
+  script 
src=https://oss.maxcdn.com/libs/html5shiv/3.7.0/html5shiv.js;/script
+  script 
src=https://oss.maxcdn.com/libs/respond.js/1.3.0/respond.min.js;/script
+  ![endif]--
+/head
+
+body
+
+script src=https://code.jquery.com/jquery.js;/script
+script 
src=//netdna.bootstrapcdn.com/bootstrap/3.0.3/js/bootstrap.min.js/script
+script src=/js/lang-tabs.js/script
+script src=/js/downloads.js/script
+
+div class=container style=max-width: 1200px;
+
+div class=masthead
+  
+p class=lead
+  a href=/
+  img src=/images/spark-logo.png
+style=height:100px; width:auto; vertical-align: bottom; margin-top: 
20px;/aspan class=tagline
+  Lightning-fast cluster computing
+  /span
+/p
+  
+/div
+
+nav class=navbar navbar-default role=navigation
+  !-- Brand and toggle get grouped for better mobile display --
+  div 

git commit: [SPARK-3984] [SPARK-3983] Fix incorrect scheduler delay and display task deserialization time in UI

2014-11-05 Thread kayousterhout
Repository: spark
Updated Branches:
  refs/heads/branch-1.2 b27d7dcaa - f4beb77f0


[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 kayousterh...@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.

(cherry picked from commit a46497eecc50f854c5c5701dc2b8a2468b76c085)
Signed-off-by: Kay Ousterhout kayousterh...@gmail.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f4beb77f
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f4beb77f
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f4beb77f

Branch: refs/heads/branch-1.2
Commit: f4beb77f083e477845b90b5049186095d2002f49
Parents: b27d7dc
Author: Kay Ousterhout kayousterh...@gmail.com
Authored: Wed Nov 5 15:30:31 2014 -0800
Committer: Kay Ousterhout kayousterh...@gmail.com
Committed: Wed Nov 5 15:30:46 2014 -0800

--
 .../org/apache/spark/executor/Executor.scala|  4 +--
 .../scala/org/apache/spark/ui/ToolTips.scala|  3 ++
 .../org/apache/spark/ui/jobs/StagePage.scala| 31 +++-
 .../spark/ui/jobs/TaskDetailsClassNames.scala   |  1 +
 4 files changed, 36 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/f4beb77f/core/src/main/scala/org/apache/spark/executor/Executor.scala
--
diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala 
b/core/src/main/scala/org/apache/spark/executor/Executor.scala
index abc1dd0..9611457 100644
--- a/core/src/main/scala/org/apache/spark/executor/Executor.scala
+++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala
@@ -161,7 +161,7 @@ private[spark] class Executor(
 }
 
 override def run() {
-  val startTime = System.currentTimeMillis()
+  val deserializeStartTime = System.currentTimeMillis()
   Thread.currentThread.setContextClassLoader(replClassLoader)
   val ser = SparkEnv.get.closureSerializer.newInstance()
   logInfo(sRunning $taskName (TID $taskId))
@@ -206,7 +206,7 @@ private[spark] class Executor(
 val afterSerialization = System.currentTimeMillis()
 
 for (m - task.metrics) {
-  m.executorDeserializeTime = taskStart - startTime
+  m.executorDeserializeTime = taskStart - deserializeStartTime
   m.executorRunTime = taskFinish - taskStart
   m.jvmGCTime = gcTime - startGCTime
   m.resultSerializationTime = afterSerialization - beforeSerialization

http://git-wip-us.apache.org/repos/asf/spark/blob/f4beb77f/core/src/main/scala/org/apache/spark/ui/ToolTips.scala
--
diff --git a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala 
b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala
index f02904d..51dc08f 100644
--- a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala
+++ b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala
@@ -24,6 +24,9 @@ private[spark] object ToolTips {
scheduler delay is large, consider decreasing the size of tasks or 
decreasing the size
of task results.
 
+  val TASK_DESERIALIZATION_TIME =
+Time spent deserializating the task closure on the executor.
+
   val INPUT = Bytes read from Hadoop or from Spark storage.
 
   val SHUFFLE_WRITE = Bytes written to disk in order to be read by a shuffle 
in a future stage.

http://git-wip-us.apache.org/repos/asf/spark/blob/f4beb77f/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
--
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala 
b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
index 7cc03b7..63ed5fc 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
@@ -114,6 +114,13 @@ private[ui] class StagePage(parent: 

git commit: SPARK-4222 [CORE] use readFully in FixedLengthBinaryRecordReader

2014-11-05 Thread matei
Repository: spark
Updated Branches:
  refs/heads/branch-1.2 f4beb77f0 - 6844e7a82


SPARK-4222 [CORE] use readFully in FixedLengthBinaryRecordReader

replaces the existing read() call with readFully().

Author: industrial-sloth industrial-sl...@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


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6844e7a8
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6844e7a8
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6844e7a8

Branch: refs/heads/branch-1.2
Commit: 6844e7a8219ac78790a422ffd5054924e7d2bea1
Parents: f4beb77
Author: industrial-sloth industrial-sl...@users.noreply.github.com
Authored: Wed Nov 5 15:38:48 2014 -0800
Committer: Matei Zaharia ma...@databricks.com
Committed: Wed Nov 5 15:38:48 2014 -0800

--
 .../org/apache/spark/input/FixedLengthBinaryRecordReader.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/6844e7a8/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala
 
b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala
index 5164a74..36a1e5d 100644
--- 
a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala
+++ 
b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala
@@ -115,7 +115,7 @@ private[spark] class FixedLengthBinaryRecordReader
 if (currentPosition  splitEnd) {
   // setup a buffer to store the record
   val buffer = recordValue.getBytes
-  fileInputStream.read(buffer, 0, recordLength)
+  fileInputStream.readFully(buffer)
   // update our current position
   currentPosition = currentPosition + recordLength
   // return true


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



git commit: SPARK-4222 [CORE] use readFully in FixedLengthBinaryRecordReader

2014-11-05 Thread matei
Repository: spark
Updated Branches:
  refs/heads/master a46497eec - f37817b18


SPARK-4222 [CORE] use readFully in FixedLengthBinaryRecordReader

replaces the existing read() call with readFully().

Author: industrial-sloth industrial-sl...@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 6844e7a8219ac78790a422ffd5054924e7d2bea1)
Signed-off-by: Matei Zaharia ma...@databricks.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f37817b1
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f37817b1
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f37817b1

Branch: refs/heads/master
Commit: f37817b18a479839b2e6118cc1cbd1059a94db52
Parents: a46497e
Author: industrial-sloth industrial-sl...@users.noreply.github.com
Authored: Wed Nov 5 15:38:48 2014 -0800
Committer: Matei Zaharia ma...@databricks.com
Committed: Wed Nov 5 15:39:16 2014 -0800

--
 .../org/apache/spark/input/FixedLengthBinaryRecordReader.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/f37817b1/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala
 
b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala
index 5164a74..36a1e5d 100644
--- 
a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala
+++ 
b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala
@@ -115,7 +115,7 @@ private[spark] class FixedLengthBinaryRecordReader
 if (currentPosition  splitEnd) {
   // setup a buffer to store the record
   val buffer = recordValue.getBytes
-  fileInputStream.read(buffer, 0, recordLength)
+  fileInputStream.readFully(buffer)
   // update our current position
   currentPosition = currentPosition + recordLength
   // return true


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



git commit: [SPARK-3797] Run external shuffle service in Yarn NM

2014-11-05 Thread andrewor14
Repository: spark
Updated Branches:
  refs/heads/branch-1.2 6844e7a82 - cf2f676f9


[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 and...@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

(cherry picked from commit 61a5cced049a8056292ba94f23fa7bd040f50685)
Signed-off-by: Andrew Or and...@databricks.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/cf2f676f
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/cf2f676f
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/cf2f676f

Branch: refs/heads/branch-1.2
Commit: cf2f676f93807bc504b77409b6c3d66f0d5e38ab
Parents: 6844e7a
Author: Andrew Or and...@databricks.com
Authored: Wed Nov 5 15:42:05 2014 -0800
Committer: Andrew Or and...@databricks.com
Committed: Wed Nov 5 15:42:17 2014 -0800

--
 .../spark/ExecutorAllocationManager.scala   |  37 ++--
 .../org/apache/spark/storage/BlockManager.scala |   8 +-
 .../scala/org/apache/spark/util/Utils.scala |  16 ++
 make-distribution.sh|   3 +
 .../network/sasl/ShuffleSecretManager.java  | 117 
 network/yarn/pom.xml|  58 ++
 .../spark/network/yarn/YarnShuffleService.java  | 176 +++
 .../network/yarn/util/HadoopConfigProvider.java |  42 +
 pom.xml |   2 +
 project/SparkBuild.scala|   8 +-
 .../spark/deploy/yarn/ExecutorRunnable.scala|  16 ++
 .../spark/deploy/yarn/ExecutorRunnable.scala|  16 ++
 12 files changed, 483 insertions(+), 16 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/cf2f676f/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala 
b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
index c11f1db..ef93009 100644
--- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
+++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
@@ -66,7 +66,6 @@ private[spark] class ExecutorAllocationManager(sc: 
SparkContext) extends Logging
   // Lower and upper bounds on the number of executors. These are required.
   private val minNumExecutors = 
conf.getInt(spark.dynamicAllocation.minExecutors, -1)
   private val maxNumExecutors = 
conf.getInt(spark.dynamicAllocation.maxExecutors, -1)
-  verifyBounds()
 
   // How long there must be backlogged tasks for before an addition is 
triggered
   private val schedulerBacklogTimeout = conf.getLong(
@@ -77,9 +76,14 @@ private[spark] class ExecutorAllocationManager(sc: 
SparkContext) extends Logging
 spark.dynamicAllocation.sustainedSchedulerBacklogTimeout, 
schedulerBacklogTimeout)
 
   // How long an executor must be idle for 

git commit: [SPARK-3797] Run external shuffle service in Yarn NM

2014-11-05 Thread andrewor14
Repository: spark
Updated Branches:
  refs/heads/master f37817b18 - 61a5cced0


[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 and...@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


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/61a5cced
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/61a5cced
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/61a5cced

Branch: refs/heads/master
Commit: 61a5cced049a8056292ba94f23fa7bd040f50685
Parents: f37817b
Author: Andrew Or and...@databricks.com
Authored: Wed Nov 5 15:42:05 2014 -0800
Committer: Andrew Or and...@databricks.com
Committed: Wed Nov 5 15:42:05 2014 -0800

--
 .../spark/ExecutorAllocationManager.scala   |  37 ++--
 .../org/apache/spark/storage/BlockManager.scala |   8 +-
 .../scala/org/apache/spark/util/Utils.scala |  16 ++
 make-distribution.sh|   3 +
 .../network/sasl/ShuffleSecretManager.java  | 117 
 network/yarn/pom.xml|  58 ++
 .../spark/network/yarn/YarnShuffleService.java  | 176 +++
 .../network/yarn/util/HadoopConfigProvider.java |  42 +
 pom.xml |   2 +
 project/SparkBuild.scala|   8 +-
 .../spark/deploy/yarn/ExecutorRunnable.scala|  16 ++
 .../spark/deploy/yarn/ExecutorRunnable.scala|  16 ++
 12 files changed, 483 insertions(+), 16 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/61a5cced/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala 
b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
index c11f1db..ef93009 100644
--- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
+++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
@@ -66,7 +66,6 @@ private[spark] class ExecutorAllocationManager(sc: 
SparkContext) extends Logging
   // Lower and upper bounds on the number of executors. These are required.
   private val minNumExecutors = 
conf.getInt(spark.dynamicAllocation.minExecutors, -1)
   private val maxNumExecutors = 
conf.getInt(spark.dynamicAllocation.maxExecutors, -1)
-  verifyBounds()
 
   // How long there must be backlogged tasks for before an addition is 
triggered
   private val schedulerBacklogTimeout = conf.getLong(
@@ -77,9 +76,14 @@ private[spark] class ExecutorAllocationManager(sc: 
SparkContext) extends Logging
 spark.dynamicAllocation.sustainedSchedulerBacklogTimeout, 
schedulerBacklogTimeout)
 
   // How long an executor must be idle for before it is removed
-  private val removeThresholdSeconds = conf.getLong(
+  private val executorIdleTimeout = conf.getLong(

git commit: SPARK-4040. Update documentation to exemplify use of local (n) value, fo...

2014-11-05 Thread matei
Repository: spark
Updated Branches:
  refs/heads/master 61a5cced0 - 868cd4c3c


SPARK-4040. Update documentation to exemplify use of local (n) value, fo...

This is a minor docs update which helps to clarify the way local[n] is used for 
streaming apps.

Author: j...@apache.org jayunit100

Closes #2964 from jayunit100/SPARK-4040 and squashes the following commits:

35b5a5e [j...@apache.org] SPARK-4040: Update documentation to exemplify use of 
local (n) value.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/868cd4c3
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/868cd4c3
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/868cd4c3

Branch: refs/heads/master
Commit: 868cd4c3ca11e6ecc4425b972d9a20c360b52425
Parents: 61a5cce
Author: j...@apache.org jayunit100
Authored: Wed Nov 5 15:45:34 2014 -0800
Committer: Matei Zaharia ma...@databricks.com
Committed: Wed Nov 5 15:45:34 2014 -0800

--
 docs/configuration.md   | 10 --
 docs/streaming-programming-guide.md | 14 +-
 2 files changed, 17 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/868cd4c3/docs/configuration.md
--
diff --git a/docs/configuration.md b/docs/configuration.md
index 685101e..0f9eb81 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -21,16 +21,22 @@ application. These properties can be set directly on a
 [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) passed to your
 `SparkContext`. `SparkConf` allows you to configure some of the common 
properties
 (e.g. master URL and application name), as well as arbitrary key-value pairs 
through the
-`set()` method. For example, we could initialize an application as follows:
+`set()` method. For example, we could initialize an application with two 
threads as follows:
+
+Note that we run with local[2], meaning two threads - which represents 
minimal parallelism, 
+which can help detect bugs that only exist when we run in a distributed 
context. 
 
 {% highlight scala %}
 val conf = new SparkConf()
- .setMaster(local)
+ .setMaster(local[2])
  .setAppName(CountingSheep)
  .set(spark.executor.memory, 1g)
 val sc = new SparkContext(conf)
 {% endhighlight %}
 
+Note that we can have more than 1 thread in local mode, and in cases like 
spark streaming, we may actually
+require one to prevent any sort of starvation issues.  
+
 ## Dynamically Loading Spark Properties
 In some cases, you may want to avoid hard-coding certain configurations in a 
`SparkConf`. For
 instance, if you'd like to run the same application with different masters or 
different

http://git-wip-us.apache.org/repos/asf/spark/blob/868cd4c3/docs/streaming-programming-guide.md
--
diff --git a/docs/streaming-programming-guide.md 
b/docs/streaming-programming-guide.md
index 8bbba88..44a1f3a 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -68,7 +68,9 @@ import org.apache.spark._
 import org.apache.spark.streaming._
 import org.apache.spark.streaming.StreamingContext._
 
-// Create a local StreamingContext with two working thread and batch interval 
of 1 second
+// Create a local StreamingContext with two working thread and batch interval 
of 1 second.
+// The master requires 2 cores to prevent from a starvation scenario.
+
 val conf = new SparkConf().setMaster(local[2]).setAppName(NetworkWordCount)
 val ssc = new StreamingContext(conf, Seconds(1))
 {% endhighlight %}
@@ -586,11 +588,13 @@ Every input DStream (except file stream) is associated 
with a single [Receiver](
 
 A receiver is run within a Spark worker/executor as a long-running task, hence 
it occupies one of the cores allocated to the Spark Streaming application. 
Hence, it is important to remember that Spark Streaming application needs to be 
allocated enough cores to process the received data, as well as, to run the 
receiver(s). Therefore, few important points to remember are:
 
-# Points to remember:
+# Points to remember
 {:.no_toc}
-- If the number of cores allocated to the application is less than or equal to 
the number of input DStreams / receivers, then the system will receive data, 
but not be able to process them.
-- When running locally, if you master URL is set to local, then there is 
only one core to run tasks.  That is insufficient for programs with even one 
input DStream (file streams are okay) as the receiver will occupy that core and 
there will be no core left to process the data.
-
+- If the number of threads allocated to the application is less than or equal 
to the number of input DStreams / receivers, then the system 

git commit: SPARK-4040. Update documentation to exemplify use of local (n) value, fo...

2014-11-05 Thread matei
Repository: spark
Updated Branches:
  refs/heads/branch-1.2 cf2f676f9 - fe4ead299


SPARK-4040. Update documentation to exemplify use of local (n) value, fo...

This is a minor docs update which helps to clarify the way local[n] is used for 
streaming apps.

Author: j...@apache.org jayunit100

Closes #2964 from jayunit100/SPARK-4040 and squashes the following commits:

35b5a5e [j...@apache.org] SPARK-4040: Update documentation to exemplify use of 
local (n) value.

(cherry picked from commit 868cd4c3ca11e6ecc4425b972d9a20c360b52425)
Signed-off-by: Matei Zaharia ma...@databricks.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/fe4ead29
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/fe4ead29
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/fe4ead29

Branch: refs/heads/branch-1.2
Commit: fe4ead2995ab8529602090ed21941b6005a07c9d
Parents: cf2f676
Author: j...@apache.org jayunit100
Authored: Wed Nov 5 15:45:34 2014 -0800
Committer: Matei Zaharia ma...@databricks.com
Committed: Wed Nov 5 15:45:43 2014 -0800

--
 docs/configuration.md   | 10 --
 docs/streaming-programming-guide.md | 14 +-
 2 files changed, 17 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/fe4ead29/docs/configuration.md
--
diff --git a/docs/configuration.md b/docs/configuration.md
index 685101e..0f9eb81 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -21,16 +21,22 @@ application. These properties can be set directly on a
 [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) passed to your
 `SparkContext`. `SparkConf` allows you to configure some of the common 
properties
 (e.g. master URL and application name), as well as arbitrary key-value pairs 
through the
-`set()` method. For example, we could initialize an application as follows:
+`set()` method. For example, we could initialize an application with two 
threads as follows:
+
+Note that we run with local[2], meaning two threads - which represents 
minimal parallelism, 
+which can help detect bugs that only exist when we run in a distributed 
context. 
 
 {% highlight scala %}
 val conf = new SparkConf()
- .setMaster(local)
+ .setMaster(local[2])
  .setAppName(CountingSheep)
  .set(spark.executor.memory, 1g)
 val sc = new SparkContext(conf)
 {% endhighlight %}
 
+Note that we can have more than 1 thread in local mode, and in cases like 
spark streaming, we may actually
+require one to prevent any sort of starvation issues.  
+
 ## Dynamically Loading Spark Properties
 In some cases, you may want to avoid hard-coding certain configurations in a 
`SparkConf`. For
 instance, if you'd like to run the same application with different masters or 
different

http://git-wip-us.apache.org/repos/asf/spark/blob/fe4ead29/docs/streaming-programming-guide.md
--
diff --git a/docs/streaming-programming-guide.md 
b/docs/streaming-programming-guide.md
index 8bbba88..44a1f3a 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -68,7 +68,9 @@ import org.apache.spark._
 import org.apache.spark.streaming._
 import org.apache.spark.streaming.StreamingContext._
 
-// Create a local StreamingContext with two working thread and batch interval 
of 1 second
+// Create a local StreamingContext with two working thread and batch interval 
of 1 second.
+// The master requires 2 cores to prevent from a starvation scenario.
+
 val conf = new SparkConf().setMaster(local[2]).setAppName(NetworkWordCount)
 val ssc = new StreamingContext(conf, Seconds(1))
 {% endhighlight %}
@@ -586,11 +588,13 @@ Every input DStream (except file stream) is associated 
with a single [Receiver](
 
 A receiver is run within a Spark worker/executor as a long-running task, hence 
it occupies one of the cores allocated to the Spark Streaming application. 
Hence, it is important to remember that Spark Streaming application needs to be 
allocated enough cores to process the received data, as well as, to run the 
receiver(s). Therefore, few important points to remember are:
 
-# Points to remember:
+# Points to remember
 {:.no_toc}
-- If the number of cores allocated to the application is less than or equal to 
the number of input DStreams / receivers, then the system will receive data, 
but not be able to process them.
-- When running locally, if you master URL is set to local, then there is 
only one core to run tasks.  That is insufficient for programs with even one 
input DStream (file streams are okay) as the receiver will occupy that core and 
there will be no core left to process the data.
-
+- If the 

git commit: SPARK-3223 runAsSparkUser cannot change HDFS write permission properly i...

2014-11-05 Thread andrewor14
Repository: spark
Updated Branches:
  refs/heads/master 868cd4c3c - f7ac8c2b1


SPARK-3223 runAsSparkUser cannot change HDFS write permission properly i...

...n mesos cluster mode

- change master newer

Author: Jongyoul Lee jongy...@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


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f7ac8c2b
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f7ac8c2b
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f7ac8c2b

Branch: refs/heads/master
Commit: f7ac8c2b1de96151231617846b7468d23379c74a
Parents: 868cd4c
Author: Jongyoul Lee jongy...@gmail.com
Authored: Wed Nov 5 15:49:42 2014 -0800
Committer: Andrew Or and...@databricks.com
Committed: Wed Nov 5 15:49:42 2014 -0800

--
 .../scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala  | 2 +-
 .../spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala  | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/f7ac8c2b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
index d8c0e2f..e4b8598 100644
--- 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
+++ 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
@@ -93,7 +93,7 @@ private[spark] class CoarseMesosSchedulerBackend(
 setDaemon(true)
 override def run() {
   val scheduler = CoarseMesosSchedulerBackend.this
-  val fwInfo = 
FrameworkInfo.newBuilder().setUser().setName(sc.appName).build()
+  val fwInfo = 
FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build()
   driver = new MesosSchedulerDriver(scheduler, fwInfo, master)
   try { {
 val ret = driver.run()

http://git-wip-us.apache.org/repos/asf/spark/blob/f7ac8c2b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
index 8e2faff..7d097a3 100644
--- 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
+++ 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
@@ -72,7 +72,7 @@ private[spark] class MesosSchedulerBackend(
 setDaemon(true)
 override def run() {
   val scheduler = MesosSchedulerBackend.this
-  val fwInfo = 
FrameworkInfo.newBuilder().setUser().setName(sc.appName).build()
+  val fwInfo = 
FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build()
   driver = new MesosSchedulerDriver(scheduler, fwInfo, master)
   try {
 val ret = driver.run()


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



git commit: SPARK-3223 runAsSparkUser cannot change HDFS write permission properly i...

2014-11-05 Thread andrewor14
Repository: spark
Updated Branches:
  refs/heads/branch-1.1 44751af9f - 590a94373


SPARK-3223 runAsSparkUser cannot change HDFS write permission properly i...

...n mesos cluster mode

- change master newer

Author: Jongyoul Lee jongy...@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

(cherry picked from commit f7ac8c2b1de96151231617846b7468d23379c74a)
Signed-off-by: Andrew Or and...@databricks.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/590a9437
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/590a9437
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/590a9437

Branch: refs/heads/branch-1.1
Commit: 590a94373d237f727f58cfd6fd02135b44d964f8
Parents: 44751af
Author: Jongyoul Lee jongy...@gmail.com
Authored: Wed Nov 5 15:49:42 2014 -0800
Committer: Andrew Or and...@databricks.com
Committed: Wed Nov 5 15:50:13 2014 -0800

--
 .../scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala  | 2 +-
 .../spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala  | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/590a9437/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
index 037fea5..43c0d8c 100644
--- 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
+++ 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
@@ -95,7 +95,7 @@ private[spark] class CoarseMesosSchedulerBackend(
 setDaemon(true)
 override def run() {
   val scheduler = CoarseMesosSchedulerBackend.this
-  val fwInfo = 
FrameworkInfo.newBuilder().setUser().setName(sc.appName).build()
+  val fwInfo = 
FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build()
   driver = new MesosSchedulerDriver(scheduler, fwInfo, master)
   try { {
 val ret = driver.run()

http://git-wip-us.apache.org/repos/asf/spark/blob/590a9437/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
index 8f064bf..ea6b06c 100644
--- 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
+++ 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
@@ -70,7 +70,7 @@ private[spark] class MesosSchedulerBackend(
 setDaemon(true)
 override def run() {
   val scheduler = MesosSchedulerBackend.this
-  val fwInfo = 
FrameworkInfo.newBuilder().setUser().setName(sc.appName).build()
+  val fwInfo = 
FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build()
   driver = new MesosSchedulerDriver(scheduler, fwInfo, master)
   try {
 val ret = driver.run()


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



git commit: SPARK-3223 runAsSparkUser cannot change HDFS write permission properly i...

2014-11-05 Thread andrewor14
Repository: spark
Updated Branches:
  refs/heads/branch-1.2 fe4ead299 - 0e16d3a3d


SPARK-3223 runAsSparkUser cannot change HDFS write permission properly i...

...n mesos cluster mode

- change master newer

Author: Jongyoul Lee jongy...@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

(cherry picked from commit f7ac8c2b1de96151231617846b7468d23379c74a)
Signed-off-by: Andrew Or and...@databricks.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0e16d3a3
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0e16d3a3
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0e16d3a3

Branch: refs/heads/branch-1.2
Commit: 0e16d3a3dde7a0988dfd8eff05922a1ac917fe28
Parents: fe4ead2
Author: Jongyoul Lee jongy...@gmail.com
Authored: Wed Nov 5 15:49:42 2014 -0800
Committer: Andrew Or and...@databricks.com
Committed: Wed Nov 5 15:50:02 2014 -0800

--
 .../scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala  | 2 +-
 .../spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala  | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0e16d3a3/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
index d8c0e2f..e4b8598 100644
--- 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
+++ 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
@@ -93,7 +93,7 @@ private[spark] class CoarseMesosSchedulerBackend(
 setDaemon(true)
 override def run() {
   val scheduler = CoarseMesosSchedulerBackend.this
-  val fwInfo = 
FrameworkInfo.newBuilder().setUser().setName(sc.appName).build()
+  val fwInfo = 
FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build()
   driver = new MesosSchedulerDriver(scheduler, fwInfo, master)
   try { {
 val ret = driver.run()

http://git-wip-us.apache.org/repos/asf/spark/blob/0e16d3a3/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
index 8e2faff..7d097a3 100644
--- 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
+++ 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
@@ -72,7 +72,7 @@ private[spark] class MesosSchedulerBackend(
 setDaemon(true)
 override def run() {
   val scheduler = MesosSchedulerBackend.this
-  val fwInfo = 
FrameworkInfo.newBuilder().setUser().setName(sc.appName).build()
+  val fwInfo = 
FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build()
   driver = new MesosSchedulerDriver(scheduler, fwInfo, master)
   try {
 val ret = driver.run()


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



git commit: [SPARK-4158] Fix for missing resources.

2014-11-05 Thread andrewor14
Repository: spark
Updated Branches:
  refs/heads/master f7ac8c2b1 - cb0eae3b7


[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 bren...@diddyinc.com

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

e5f9580 [Brenden Matthews] [SPARK-4158] Fix for missing resources.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/cb0eae3b
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/cb0eae3b
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/cb0eae3b

Branch: refs/heads/master
Commit: cb0eae3b78d7f6f56c0b9521ee48564a4967d3de
Parents: f7ac8c2
Author: Brenden Matthews bren...@diddyinc.com
Authored: Wed Nov 5 16:02:44 2014 -0800
Committer: Andrew Or and...@databricks.com
Committed: Wed Nov 5 16:02:44 2014 -0800

--
 .../scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala | 3 +--
 .../spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala | 3 +--
 2 files changed, 2 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/cb0eae3b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
index e4b8598..5289661 100644
--- 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
+++ 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
@@ -242,8 +242,7 @@ private[spark] class CoarseMesosSchedulerBackend(
 for (r - res if r.getName == name) {
   return r.getScalar.getValue
 }
-// If we reached here, no resource with the required name was present
-throw new IllegalArgumentException(No resource called  + name +  in  + 
res)
+0
   }
 
   /** Build a Mesos resource protobuf object */

http://git-wip-us.apache.org/repos/asf/spark/blob/cb0eae3b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
index 7d097a3..c5f3493 100644
--- 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
+++ 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
@@ -278,8 +278,7 @@ private[spark] class MesosSchedulerBackend(
 for (r - res if r.getName == name) {
   return r.getScalar.getValue
 }
-// If we reached here, no resource with the required name was present
-throw new IllegalArgumentException(No resource called  + name +  in  + 
res)
+0
   }
 
   /** Turn a Spark TaskDescription into a Mesos task */


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



git commit: [SPARK-4158] Fix for missing resources.

2014-11-05 Thread andrewor14
Repository: spark
Updated Branches:
  refs/heads/branch-1.2 0e16d3a3d - 9ac5c517b


[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 bren...@diddyinc.com

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

e5f9580 [Brenden Matthews] [SPARK-4158] Fix for missing resources.

(cherry picked from commit cb0eae3b78d7f6f56c0b9521ee48564a4967d3de)
Signed-off-by: Andrew Or and...@databricks.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9ac5c517
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9ac5c517
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9ac5c517

Branch: refs/heads/branch-1.2
Commit: 9ac5c517b64606db7d6b8ac3b823c3d5a45e0ed0
Parents: 0e16d3a
Author: Brenden Matthews bren...@diddyinc.com
Authored: Wed Nov 5 16:02:44 2014 -0800
Committer: Andrew Or and...@databricks.com
Committed: Wed Nov 5 16:02:53 2014 -0800

--
 .../scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala | 3 +--
 .../spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala | 3 +--
 2 files changed, 2 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/9ac5c517/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
index e4b8598..5289661 100644
--- 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
+++ 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
@@ -242,8 +242,7 @@ private[spark] class CoarseMesosSchedulerBackend(
 for (r - res if r.getName == name) {
   return r.getScalar.getValue
 }
-// If we reached here, no resource with the required name was present
-throw new IllegalArgumentException(No resource called  + name +  in  + 
res)
+0
   }
 
   /** Build a Mesos resource protobuf object */

http://git-wip-us.apache.org/repos/asf/spark/blob/9ac5c517/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
index 7d097a3..c5f3493 100644
--- 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
+++ 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
@@ -278,8 +278,7 @@ private[spark] class MesosSchedulerBackend(
 for (r - res if r.getName == name) {
   return r.getScalar.getValue
 }
-// If we reached here, no resource with the required name was present
-throw new IllegalArgumentException(No resource called  + name +  in  + 
res)
+0
   }
 
   /** Turn a Spark TaskDescription into a Mesos task */


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



git commit: [SPARK-4158] Fix for missing resources.

2014-11-05 Thread andrewor14
Repository: spark
Updated Branches:
  refs/heads/branch-1.1 590a94373 - c58c1bb83


[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 bren...@diddyinc.com

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

e5f9580 [Brenden Matthews] [SPARK-4158] Fix for missing resources.

(cherry picked from commit cb0eae3b78d7f6f56c0b9521ee48564a4967d3de)
Signed-off-by: Andrew Or and...@databricks.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c58c1bb8
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c58c1bb8
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c58c1bb8

Branch: refs/heads/branch-1.1
Commit: c58c1bb830c9efdecd680d19228dca56362b90b5
Parents: 590a943
Author: Brenden Matthews bren...@diddyinc.com
Authored: Wed Nov 5 16:02:44 2014 -0800
Committer: Andrew Or and...@databricks.com
Committed: Wed Nov 5 16:03:06 2014 -0800

--
 .../scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala | 3 +--
 .../spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala | 3 +--
 2 files changed, 2 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/c58c1bb8/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
index 43c0d8c..9cdde20 100644
--- 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
+++ 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
@@ -235,8 +235,7 @@ private[spark] class CoarseMesosSchedulerBackend(
 for (r - res if r.getName == name) {
   return r.getScalar.getValue
 }
-// If we reached here, no resource with the required name was present
-throw new IllegalArgumentException(No resource called  + name +  in  + 
res)
+0
   }
 
   /** Build a Mesos resource protobuf object */

http://git-wip-us.apache.org/repos/asf/spark/blob/c58c1bb8/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
index ea6b06c..a98d09d 100644
--- 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
+++ 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
@@ -273,8 +273,7 @@ private[spark] class MesosSchedulerBackend(
 for (r - res if r.getName == name) {
   return r.getScalar.getValue
 }
-// If we reached here, no resource with the required name was present
-throw new IllegalArgumentException(No resource called  + name +  in  + 
res)
+0
   }
 
   /** Turn a Spark TaskDescription into a Mesos task */


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



git commit: [SPARK-4254] [mllib] MovieLensALS bug fix

2014-11-05 Thread meng
Repository: spark
Updated Branches:
  refs/heads/master cb0eae3b7 - c315d1316


[SPARK-4254] [mllib] MovieLensALS bug fix

Changed code so it does not try to serialize Params.
CC: mengxr  debasish83 srowen

Author: Joseph K. Bradley jos...@databricks.com

Closes #3116 from jkbradley/als-bugfix and squashes the following commits:

e575bd8 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into 
als-bugfix
9401b16 [Joseph K. Bradley] changed implicitPrefs so it is not serialized to 
fix MovieLensALS example bug


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c315d131
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c315d131
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c315d131

Branch: refs/heads/master
Commit: c315d1316cb2372e90ae3a12f72d5b3304435a6b
Parents: cb0eae3
Author: Joseph K. Bradley jos...@databricks.com
Authored: Wed Nov 5 19:51:18 2014 -0800
Committer: Xiangrui Meng m...@databricks.com
Committed: Wed Nov 5 19:51:18 2014 -0800

--
 .../scala/org/apache/spark/examples/mllib/MovieLensALS.scala | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/c315d131/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala
--
diff --git 
a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala 
b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala
index 8796c28..91a0a86 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala
@@ -106,9 +106,11 @@ object MovieLensALS {
 
 Logger.getRootLogger.setLevel(Level.WARN)
 
+val implicitPrefs = params.implicitPrefs
+
 val ratings = sc.textFile(params.input).map { line =
   val fields = line.split(::)
-  if (params.implicitPrefs) {
+  if (implicitPrefs) {
 /*
  * MovieLens ratings are on a scale of 1-5:
  * 5: Must see


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



git commit: [SPARK-4254] [mllib] MovieLensALS bug fix

2014-11-05 Thread meng
Repository: spark
Updated Branches:
  refs/heads/branch-1.2 9ac5c517b - ff84a8ae2


[SPARK-4254] [mllib] MovieLensALS bug fix

Changed code so it does not try to serialize Params.
CC: mengxr  debasish83 srowen

Author: Joseph K. Bradley jos...@databricks.com

Closes #3116 from jkbradley/als-bugfix and squashes the following commits:

e575bd8 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into 
als-bugfix
9401b16 [Joseph K. Bradley] changed implicitPrefs so it is not serialized to 
fix MovieLensALS example bug

(cherry picked from commit c315d1316cb2372e90ae3a12f72d5b3304435a6b)
Signed-off-by: Xiangrui Meng m...@databricks.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ff84a8ae
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ff84a8ae
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ff84a8ae

Branch: refs/heads/branch-1.2
Commit: ff84a8ae258083423529885d85bf1d939a62d899
Parents: 9ac5c51
Author: Joseph K. Bradley jos...@databricks.com
Authored: Wed Nov 5 19:51:18 2014 -0800
Committer: Xiangrui Meng m...@databricks.com
Committed: Wed Nov 5 19:51:26 2014 -0800

--
 .../scala/org/apache/spark/examples/mllib/MovieLensALS.scala | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/ff84a8ae/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala
--
diff --git 
a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala 
b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala
index 8796c28..91a0a86 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala
@@ -106,9 +106,11 @@ object MovieLensALS {
 
 Logger.getRootLogger.setLevel(Level.WARN)
 
+val implicitPrefs = params.implicitPrefs
+
 val ratings = sc.textFile(params.input).map { line =
   val fields = line.split(::)
-  if (params.implicitPrefs) {
+  if (implicitPrefs) {
 /*
  * MovieLens ratings are on a scale of 1-5:
  * 5: Must see


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



git commit: [SPARK-4262][SQL] add .schemaRDD to JavaSchemaRDD

2014-11-05 Thread meng
Repository: spark
Updated Branches:
  refs/heads/master c315d1316 - 3d2b5bc5b


[SPARK-4262][SQL] add .schemaRDD to JavaSchemaRDD

 marmbrus

Author: Xiangrui Meng m...@databricks.com

Closes #3125 from mengxr/SPARK-4262 and squashes the following commits:

307695e [Xiangrui Meng] add .schemaRDD to JavaSchemaRDD


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3d2b5bc5
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3d2b5bc5
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3d2b5bc5

Branch: refs/heads/master
Commit: 3d2b5bc5bb979d8b0b71e06bc0f4548376fdbb98
Parents: c315d13
Author: Xiangrui Meng m...@databricks.com
Authored: Wed Nov 5 19:56:16 2014 -0800
Committer: Xiangrui Meng m...@databricks.com
Committed: Wed Nov 5 19:56:16 2014 -0800

--
 .../main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala  | 3 +++
 1 file changed, 3 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3d2b5bc5/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala
index 1e0ccb3..78e8d90 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala
@@ -47,6 +47,9 @@ class JavaSchemaRDD(
 
   private[sql] val baseSchemaRDD = new SchemaRDD(sqlContext, logicalPlan)
 
+  /** Returns the underlying Scala SchemaRDD. */
+  val schemaRDD: SchemaRDD = baseSchemaRDD
+
   override val classTag = scala.reflect.classTag[Row]
 
   override def wrapRDD(rdd: RDD[Row]): JavaRDD[Row] = JavaRDD.fromRDD(rdd)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



git commit: [SPARK-4262][SQL] add .schemaRDD to JavaSchemaRDD

2014-11-05 Thread meng
Repository: spark
Updated Branches:
  refs/heads/branch-1.2 ff84a8ae2 - 7e0da9f6b


[SPARK-4262][SQL] add .schemaRDD to JavaSchemaRDD

 marmbrus

Author: Xiangrui Meng m...@databricks.com

Closes #3125 from mengxr/SPARK-4262 and squashes the following commits:

307695e [Xiangrui Meng] add .schemaRDD to JavaSchemaRDD

(cherry picked from commit 3d2b5bc5bb979d8b0b71e06bc0f4548376fdbb98)
Signed-off-by: Xiangrui Meng m...@databricks.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7e0da9f6
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7e0da9f6
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7e0da9f6

Branch: refs/heads/branch-1.2
Commit: 7e0da9f6b423842adc9fed2db2d4a80cab541351
Parents: ff84a8a
Author: Xiangrui Meng m...@databricks.com
Authored: Wed Nov 5 19:56:16 2014 -0800
Committer: Xiangrui Meng m...@databricks.com
Committed: Wed Nov 5 19:56:24 2014 -0800

--
 .../main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala  | 3 +++
 1 file changed, 3 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/7e0da9f6/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala
index 1e0ccb3..78e8d90 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala
@@ -47,6 +47,9 @@ class JavaSchemaRDD(
 
   private[sql] val baseSchemaRDD = new SchemaRDD(sqlContext, logicalPlan)
 
+  /** Returns the underlying Scala SchemaRDD. */
+  val schemaRDD: SchemaRDD = baseSchemaRDD
+
   override val classTag = scala.reflect.classTag[Row]
 
   override def wrapRDD(rdd: RDD[Row]): JavaRDD[Row] = JavaRDD.fromRDD(rdd)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



git commit: [SPARK-4137] [EC2] Don't change working dir on user

2014-11-05 Thread shivaram
Repository: spark
Updated Branches:
  refs/heads/master 3d2b5bc5b - db45f5ad0


[SPARK-4137] [EC2] Don't change working dir on user

This issue was uncovered after [this 
discussion](https://issues.apache.org/jira/browse/SPARK-3398?focusedCommentId=14187471page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14187471).

Don't change the working directory on the user. This breaks relative paths the 
user may pass in, e.g., for the SSH identity file.

```
./ec2/spark-ec2 -i ../my.pem
```

This patch will preserve the user's current working directory and allow calls 
like the one above to work.

Author: Nicholas Chammas nicholas.cham...@gmail.com

Closes #2988 from nchammas/spark-ec2-cwd and squashes the following commits:

f3850b5 [Nicholas Chammas] pep8 fix
fbc20c7 [Nicholas Chammas] revert to old commenting style
752f958 [Nicholas Chammas] specify deploy.generic path absolutely
bcdf6a5 [Nicholas Chammas] fix typo
77871a2 [Nicholas Chammas] add clarifying comment
ce071fc [Nicholas Chammas] don't change working dir


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/db45f5ad
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/db45f5ad
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/db45f5ad

Branch: refs/heads/master
Commit: db45f5ad0368760dbeaa618a04f66ae9b2bed656
Parents: 3d2b5bc
Author: Nicholas Chammas nicholas.cham...@gmail.com
Authored: Wed Nov 5 20:45:35 2014 -0800
Committer: Shivaram Venkataraman shiva...@cs.berkeley.edu
Committed: Wed Nov 5 20:45:35 2014 -0800

--
 ec2/spark-ec2|  8 ++--
 ec2/spark_ec2.py | 12 +++-
 2 files changed, 17 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/db45f5ad/ec2/spark-ec2
--
diff --git a/ec2/spark-ec2 b/ec2/spark-ec2
index 31f9771..4aa9082 100755
--- a/ec2/spark-ec2
+++ b/ec2/spark-ec2
@@ -18,5 +18,9 @@
 # limitations under the License.
 #
 
-cd `dirname $0`
-PYTHONPATH=./third_party/boto-2.4.1.zip/boto-2.4.1:$PYTHONPATH python 
./spark_ec2.py $@
+# Preserve the user's CWD so that relative paths are passed correctly to 
+#+ the underlying Python script.
+SPARK_EC2_DIR=$(dirname $0)
+
+PYTHONPATH=${SPARK_EC2_DIR}/third_party/boto-2.4.1.zip/boto-2.4.1:$PYTHONPATH
 \
+python ${SPARK_EC2_DIR}/spark_ec2.py $@

http://git-wip-us.apache.org/repos/asf/spark/blob/db45f5ad/ec2/spark_ec2.py
--
diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py
index 50f88f7..a5396c2 100755
--- a/ec2/spark_ec2.py
+++ b/ec2/spark_ec2.py
@@ -40,6 +40,7 @@ from boto.ec2.blockdevicemapping import BlockDeviceMapping, 
BlockDeviceType, EBS
 from boto import ec2
 
 DEFAULT_SPARK_VERSION = 1.1.0
+SPARK_EC2_DIR = os.path.dirname(os.path.realpath(__file__))
 
 MESOS_SPARK_EC2_BRANCH = v4
 # A URL prefix from which to fetch AMI information
@@ -593,7 +594,14 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, 
deploy_ssh_key):
 )
 
 print Deploying files to master...
-deploy_files(conn, deploy.generic, opts, master_nodes, slave_nodes, 
modules)
+deploy_files(
+conn=conn,
+root_dir=SPARK_EC2_DIR + / + deploy.generic,
+opts=opts,
+master_nodes=master_nodes,
+slave_nodes=slave_nodes,
+modules=modules
+)
 
 print Running setup on master...
 setup_spark_cluster(master, opts)
@@ -730,6 +738,8 @@ def get_num_disks(instance_type):
 # cluster (e.g. lists of masters and slaves). Files are only deployed to
 # the first master instance in the cluster, and we expect the setup
 # script to be run on that instance to copy them to other nodes.
+#
+# root_dir should be an absolute path to the directory with the files we want 
to deploy.
 def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules):
 active_master = master_nodes[0].public_dns_name
 


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



git commit: [SPARK-4137] [EC2] Don't change working dir on user

2014-11-05 Thread shivaram
Repository: spark
Updated Branches:
  refs/heads/branch-1.2 7e0da9f6b - 70f6f36e0


[SPARK-4137] [EC2] Don't change working dir on user

This issue was uncovered after [this 
discussion](https://issues.apache.org/jira/browse/SPARK-3398?focusedCommentId=14187471page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14187471).

Don't change the working directory on the user. This breaks relative paths the 
user may pass in, e.g., for the SSH identity file.

```
./ec2/spark-ec2 -i ../my.pem
```

This patch will preserve the user's current working directory and allow calls 
like the one above to work.

Author: Nicholas Chammas nicholas.cham...@gmail.com

Closes #2988 from nchammas/spark-ec2-cwd and squashes the following commits:

f3850b5 [Nicholas Chammas] pep8 fix
fbc20c7 [Nicholas Chammas] revert to old commenting style
752f958 [Nicholas Chammas] specify deploy.generic path absolutely
bcdf6a5 [Nicholas Chammas] fix typo
77871a2 [Nicholas Chammas] add clarifying comment
ce071fc [Nicholas Chammas] don't change working dir

(cherry picked from commit db45f5ad0368760dbeaa618a04f66ae9b2bed656)
Signed-off-by: Shivaram Venkataraman shiva...@cs.berkeley.edu


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/70f6f36e
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/70f6f36e
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/70f6f36e

Branch: refs/heads/branch-1.2
Commit: 70f6f36e03f97847cd2f3e4fe2902bb8459ca6a3
Parents: 7e0da9f
Author: Nicholas Chammas nicholas.cham...@gmail.com
Authored: Wed Nov 5 20:45:35 2014 -0800
Committer: Shivaram Venkataraman shiva...@cs.berkeley.edu
Committed: Wed Nov 5 20:45:55 2014 -0800

--
 ec2/spark-ec2|  8 ++--
 ec2/spark_ec2.py | 12 +++-
 2 files changed, 17 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/70f6f36e/ec2/spark-ec2
--
diff --git a/ec2/spark-ec2 b/ec2/spark-ec2
index 31f9771..4aa9082 100755
--- a/ec2/spark-ec2
+++ b/ec2/spark-ec2
@@ -18,5 +18,9 @@
 # limitations under the License.
 #
 
-cd `dirname $0`
-PYTHONPATH=./third_party/boto-2.4.1.zip/boto-2.4.1:$PYTHONPATH python 
./spark_ec2.py $@
+# Preserve the user's CWD so that relative paths are passed correctly to 
+#+ the underlying Python script.
+SPARK_EC2_DIR=$(dirname $0)
+
+PYTHONPATH=${SPARK_EC2_DIR}/third_party/boto-2.4.1.zip/boto-2.4.1:$PYTHONPATH
 \
+python ${SPARK_EC2_DIR}/spark_ec2.py $@

http://git-wip-us.apache.org/repos/asf/spark/blob/70f6f36e/ec2/spark_ec2.py
--
diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py
index 50f88f7..a5396c2 100755
--- a/ec2/spark_ec2.py
+++ b/ec2/spark_ec2.py
@@ -40,6 +40,7 @@ from boto.ec2.blockdevicemapping import BlockDeviceMapping, 
BlockDeviceType, EBS
 from boto import ec2
 
 DEFAULT_SPARK_VERSION = 1.1.0
+SPARK_EC2_DIR = os.path.dirname(os.path.realpath(__file__))
 
 MESOS_SPARK_EC2_BRANCH = v4
 # A URL prefix from which to fetch AMI information
@@ -593,7 +594,14 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, 
deploy_ssh_key):
 )
 
 print Deploying files to master...
-deploy_files(conn, deploy.generic, opts, master_nodes, slave_nodes, 
modules)
+deploy_files(
+conn=conn,
+root_dir=SPARK_EC2_DIR + / + deploy.generic,
+opts=opts,
+master_nodes=master_nodes,
+slave_nodes=slave_nodes,
+modules=modules
+)
 
 print Running setup on master...
 setup_spark_cluster(master, opts)
@@ -730,6 +738,8 @@ def get_num_disks(instance_type):
 # cluster (e.g. lists of masters and slaves). Files are only deployed to
 # the first master instance in the cluster, and we expect the setup
 # script to be run on that instance to copy them to other nodes.
+#
+# root_dir should be an absolute path to the directory with the files we want 
to deploy.
 def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules):
 active_master = master_nodes[0].public_dns_name
 


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org