Re: Should new YARN shuffle service work with "yarn-alpha"?

2014-11-07 Thread Sean Owen
Hm. Problem is, core depends directly on it: [error] /Users/srowen/Documents/spark/core/src/main/scala/org/apache/spark/SecurityManager.scala:25: object sasl is not a member of package org.apache.spark.network [error] import org.apache.spark.network.sasl.SecretKeyHolder [error]

Re: Should new YARN shuffle service work with "yarn-alpha"?

2014-11-07 Thread Patrick Wendell
I bet it doesn't work. +1 on isolating it's inclusion to only the newer YARN API's. - Patrick On Fri, Nov 7, 2014 at 11:43 PM, Sean Owen wrote: > I noticed that this doesn't compile: > > mvn -Pyarn-alpha -Phadoop-0.23 -Dhadoop.version=0.23.7 -DskipTests clean > package > > [error] warning: [opt

Should new YARN shuffle service work with "yarn-alpha"?

2014-11-07 Thread Sean Owen
I noticed that this doesn't compile: mvn -Pyarn-alpha -Phadoop-0.23 -Dhadoop.version=0.23.7 -DskipTests clean package [error] warning: [options] bootstrap class path not set in conjunction with -source 1.6 [error] /Users/srowen/Documents/spark/network/yarn/src/main/java/org/apache/spark/network/

Re: [MLlib] Contributing Algorithm for Outlier Detection

2014-11-07 Thread Mayur Rustagi
> > We should take a vector instead giving the user flexibility to decide > data source/ type What do you mean by vector datatype exactly? Mayur Rustagi Ph: +1 (760) 203 3257 http://www.sigmoidanalytics.com @mayur_rustagi On Wed, Nov 5, 2014 at 6:45 AM, slccl

Re: Replacing Spark's native scheduler with Sparrow

2014-11-07 Thread Nicholas Chammas
Yeah, perhaps I misunderstood what Michael was saying. But thanks for pointing out the relevant UI functionality. On Sat, Nov 8, 2014 at 1:22 AM, Kay Ousterhout wrote: > I don't have much more info than what Shivaram said. My sense is that, > over time, task launch overhead with Spark has slowl

Re: Replacing Spark's native scheduler with Sparrow

2014-11-07 Thread Kay Ousterhout
I don't have much more info than what Shivaram said. My sense is that, over time, task launch overhead with Spark has slowly grown as Spark supports more and more functionality. However, I haven't seen it be as high as the 100ms Michael quoted (maybe this was for jobs with tasks that have much la

Re: [VOTE] Designating maintainers for some Spark components

2014-11-07 Thread vaquar khan
+1 (binding) On 8 Nov 2014 07:26, "Davies Liu" wrote: > Sorry for my last email, I misunderstood the proposal here, all the > committer still have equal -1 to all the code changes. > > Also, as mentioned in the proposal, the sign off only happens to > public API and architect, something like disc

Re: Replacing Spark's native scheduler with Sparrow

2014-11-07 Thread Shivaram Venkataraman
I think Kay might be able to give a better answer. The most recent benchmark I remember had the number at at somewhere between 8.6ms and 14.6ms depending on the Spark version ( https://github.com/apache/spark/pull/2030#issuecomment-52715181). Another point to note is that this is the total time to

Re: Replacing Spark's native scheduler with Sparrow

2014-11-07 Thread Nicholas Chammas
Hmm, relevant quote from section 3.3: newer frameworks like Spark [35] reduce the overhead to 5ms. To support > tasks that complete in hundreds of mil- liseconds, we argue for reducing > task launch overhead even further to 1ms so that launch overhead > constitutes at most 1% of task runtime. By m

Re: Replacing Spark's native scheduler with Sparrow

2014-11-07 Thread Shivaram Venkataraman
On Fri, Nov 7, 2014 at 8:04 PM, Nicholas Chammas wrote: > Sounds good. I'm looking forward to tracking improvements in this area. > > Also, just to connect some more dots here, I just remembered that there is > currently an initiative to add an IndexedRDD >

Re: Replacing Spark's native scheduler with Sparrow

2014-11-07 Thread Nicholas Chammas
Sounds good. I'm looking forward to tracking improvements in this area. Also, just to connect some more dots here, I just remembered that there is currently an initiative to add an IndexedRDD interface. Some interesting use cases mentioned there i

Re: Replacing Spark's native scheduler with Sparrow

2014-11-07 Thread Kay Ousterhout
On Fri, Nov 7, 2014 at 6:20 PM, Nicholas Chammas wrote: > If, for example, you have a cluster of 100 machines, this means the >> scheduler can launch 150 tasks per machine per second. > > > Did you mean 15 tasks per machine per second here? Or alternatively, 10 > machines? > Yes -- sorry for the

Re: Replacing Spark's native scheduler with Sparrow

2014-11-07 Thread Nicholas Chammas
> > If, for example, you have a cluster of 100 machines, this means the > scheduler can launch 150 tasks per machine per second. Did you mean 15 tasks per machine per second here? Or alternatively, 10 machines? I don't know of any existing Spark clusters that have a large enough number > of mach

Re: [VOTE] Designating maintainers for some Spark components

2014-11-07 Thread Davies Liu
Sorry for my last email, I misunderstood the proposal here, all the committer still have equal -1 to all the code changes. Also, as mentioned in the proposal, the sign off only happens to public API and architect, something like discussion about code style things are still the same. So, I'd rever

Re: [VOTE] Designating maintainers for some Spark components

2014-11-07 Thread Tathagata Das
+1 (binding) I agree with the proposal that it just formalizes what we have been doing till now, and will increase the efficiency and focus of the review process. To address Davies' concern, I agree coding style is often a hot topic of contention. But that is just an indication that our processes

Re: Replacing Spark's native scheduler with Sparrow

2014-11-07 Thread Kay Ousterhout
Hi Nick, This hasn't yet been directly supported by Spark because of a lack of demand. The last time I ran a throughput test on the default Spark scheduler (~1 year ago, so this may have changed), it could launch approximately 1500 tasks / second. If, for example, you have a cluster of 100 machi

Re: [VOTE] Designating maintainers for some Spark components

2014-11-07 Thread Davies Liu
-1 (not binding, +1 for maintainer, -1 for sign off) Agree with Greg and Vinod. In the beginning, everything is better (more efficient, more focus), but after some time, fighting begins. Code style is the most hot topic to fight (we already saw it in some PRs). If two committers (one of them is m

Replacing Spark's native scheduler with Sparrow

2014-11-07 Thread Nicholas Chammas
I just watched Kay's talk from 2013 on Sparrow . Is replacing Spark's native scheduler with Sparrow still on the books? The Sparrow repo hasn't been updated recently, and I don't see any JIRA issues about it. It woul

Bind exception while running FlumeEventCount

2014-11-07 Thread Jeniba Johnson
Hi, I have installed spark-1.1.0 and apache flume 1.4 for running streaming example FlumeEventCount. Previously the code was working fine. Now Iam facing with the below mentioned issues. My flume is running properly it is able to write the file. The command I use is bin/run-example org.apac

Re: Implementing TinkerPop on top of GraphX

2014-11-07 Thread Kushal Datta
I think if we are going to use GraphX as the query engine in Tinkerpop3, then the Tinkerpop3 community is the right platform to further the discussion. The reason I asked the question on improving APIs in GraphX is because why only Gremlin, any graph DSL can exploit the GraphX APIs. Cypher has som

Re: Implementing TinkerPop on top of GraphX

2014-11-07 Thread York, Brennon
I’m definitely onboard to help / take a portion of this work. I too am wondering what the proper discussion venue should be moving forward given Reynold’s remarks on a community project hosted outside Spark. If I’m understanding correctly my take would be: 1. to find a core group of developers

Re: Implementing TinkerPop on top of GraphX

2014-11-07 Thread Kyle Ellrott
Who here would be interested in helping to work on an implementation of the Tikerpop3 Gremlin API for Spark? Is this something that should continue in the Spark discussion group, or should it migrate to the Gremlin message group? Reynold is right that there will be inherent mismatches in the APIs,

Re: [VOTE] Designating maintainers for some Spark components

2014-11-07 Thread Kay Ousterhout
+1 (binding) I see this as a way to increase transparency and efficiency around a process that already informally exists, with benefits to both new contributors and committers. For new contributors, it makes clear who they should ping about a pending patch. For committers, it's a good reference

RE: Appropriate way to add a debug flag

2014-11-07 Thread Ganelin, Ilya
That perspective - identifying the non serializable components - is actually very helpful. I'll look into figuring out a way to do this. Thanks ! -Original Message- From: Josh Rosen [rosenvi...@gmail.com] Sent: Friday, November 07, 2014 12:51 PM Eastern Standa

Re: Appropriate way to add a debug flag

2014-11-07 Thread Josh Rosen
(Whoops, forgot to copy dev@ in my original reply; adding it back) Yeah, the GraphViz part was mostly for fun and for understanding cyclic object graphs. In general, an object graph might contain cycles, so for understanding the overall structure it's handy to have a picture. The GraphViz thing

Re: How spark/*/Storage/BlockManagerMaster.askDriverWithReply() responds to various query messages

2014-11-07 Thread Imran Rashid
ask() is a method on every Actor. It comes from the akka library, which spark uses for a lot of the communication between various components. There is some documentation on ask() here (go to the section on "Send messages"): http://doc.akka.io/docs/akka/2.2.3/scala/actors.html though if you are t

How spark/*/Storage/BlockManagerMaster.askDriverWithReply() responds to various query messages

2014-11-07 Thread rapelly kartheek
Hi, I am trying to understand how the /spark/*/Storage/BlockManagerMaster.askDriverWithReply() works. def getPeers(blockManagerId: BlockManagerId, numPeers: Int): Seq[BlockManagerId] = { val result = askDriverWithReply[Seq[BlockManagerId]](GetPeers(blockManagerId, numPeers)) if (result.length

Re: proposal / discuss: multiple Serializers within a SparkContext?

2014-11-07 Thread Matei Zaharia
Yup, the JIRA for this was https://issues.apache.org/jira/browse/SPARK-540 (one of our older JIRAs). I think it would be interesting to explore this further. Basically the way to add it into the API would be to add a version of persist() that takes another class than StorageLevel, say StorageStr

Re: proposal / discuss: multiple Serializers within a SparkContext?

2014-11-07 Thread Reynold Xin
Technically you can already do custom serializer for each shuffle operation (it is part of the ShuffledRDD). I've seen Matei suggesting on jira issues (or github) in the past a "storage policy" in which you can specify how data should be stored. I think that would be a great API to have in the long

proposal / discuss: multiple Serializers within a SparkContext?

2014-11-07 Thread Sandy Ryza
Hey all, Was messing around with Spark and Google FlatBuffers for fun, and it got me thinking about Spark and serialization. I know there's been work / talk about in-memory columnar formats Spark SQL, so maybe there are ways to provide this flexibility already that I've missed? Either way, my th