Re: Set memory when using local[k]

2014-02-23 Thread Aaron Davidson
local[k] simply creates k threads in the same process, so you just need to set the memory of the spawning process, using standard Java options. Also, avoid SPARK_MEM when possible, as it will be deprecated by 1.0.0. (In 0.9.0 and prior versions, you still need SPARK_MEM to set shell memory, though

Re: Spark High Availability

2014-02-22 Thread Aaron Davidson
by using a dns record or haproxy or some other > software? > On Feb 23, 2014 3:51 AM, "Aaron Davidson" wrote: > >> The current way of solving this problem is to list all three masters as >> your master url; e.g.,: >> spark://host1:port1,host2:port2,host3:port

Re: Spark High Availability

2014-02-22 Thread Aaron Davidson
The current way of solving this problem is to list all three masters as your master url; e.g.,: spark://host1:port1,host2:port2,host3:port3 This will try all three in parallel and use whichever one is currently the master. This should work as long as you don't have to introduce a new node as a bac

Re: Why collect() has a stage but first() not?

2014-02-19 Thread Aaron Davidson
my RDD is placed on the worker nodes. So how can driver perform the > action by itself? > > > On Wed, Feb 19, 2014 at 10:57 AM, Aaron Davidson wrote: > >> first() is allowed to "run locally", which means that the driver will >> execute the action itself without l

Re: Why collect() has a stage but first() not?

2014-02-19 Thread Aaron Davidson
first() is allowed to "run locally", which means that the driver will execute the action itself without launching any tasks. This is also true of take(n) for sufficiently small n, for instance. On Wed, Feb 19, 2014 at 9:55 AM, David Thomas wrote: > If I perform a 'collect' action on the RDD, I

Re: Using local[N] gets "Too many open files"?

2014-02-16 Thread Aaron Davidson
If you are intentionally opening many files at once and getting that error, then it is a fixable OS issue. Please check out this discussion regarding changing the file limit in /etc/limits.conf: http://apache-spark-user-list.1001560.n3.nabble.com/Too-many-open-files-td1464.html If you feel that yo

Re: [External] Re: Too many open files

2014-02-13 Thread Aaron Davidson
Note that you may just be changing the soft limit, and are still being capped by the hard (system-wide) limit. Changing the /etc/limit.conf file as specified above allows you to modify both the soft and hard limits, and requires a restart of the machine to take effect. On Thu, Feb 13, 2014 at 11:

Re: Spark Release 0.9.0 missing org.apache.spark.streaming package + misleading documentation on http://spark.incubator.apache.org/releases/spark-release-0-9-0.html

2014-02-12 Thread Aaron Davidson
Be sure to link against the 0.9.0 version, though! Here are the updated docs: https://spark.incubator.apache.org/docs/0.9.0/streaming-programming-guide.html#basics On Wed, Feb 12, 2014 at 2:48 PM, Evan R. Sparks wrote: > You'll need to link against the spark-streaming artifact as well - see the

Re: Shuffle file not found Exception

2014-02-09 Thread Aaron Davidson
This sounds bad, and probably related to shuffle file consolidation. Turning off consolidation would probably get you working again, but I'd really love to track down the bug. Do you know if any tasks fail before those errors start occurring? It's very possible that another exception is occurring w

Re: In Memory Caching blowing up the size

2014-02-07 Thread Aaron Davidson
Have you tried caching the RDD in memory with serialization? How are you measuring the in-memory size? In general I can imagine a blowup of 2-3 times for small rows would be expected, but 10x does seem excessive. On Fri, Feb 7, 2014 at 12:38 PM, Vipul Pandey wrote: > Hi, > > I have a very smal

Re: Spark app gets slower as it gets executed more times

2014-02-06 Thread Aaron Davidson
2014 at 5:58 AM, Aureliano Buendia wrote: > > > > On Mon, Feb 3, 2014 at 12:26 AM, Aaron Davidson wrote: > >> Are you seeing any exceptions in between running apps? Does restarting >> the master/workers actually cause Spark to speed back up again? It's >> possi

Re: Hash Join in Spark

2014-02-03 Thread Aaron Davidson
Regarding your initial question, I am not certain exactly what you mean by the properties you mention. Here are some details about the hash join that at least use those words, but perhaps not as you intended them. The hash join is blocking in the sense that we will aggregate all inputs to a partit

Re: Hash Join in Spark

2014-02-03 Thread Aaron Davidson
This method is doing very little. Line 2 constructs the CoGroupedRDD, which will do all the real work. Note that while this cogroup function just groups 2 RDDs together, CoGroupedRDD allows general n-way cogrouping, so it takes a Seq[RDD(K, _)] rather than just 2 such key-value RDDs. The rest of t

Re: default parallelism in trunk

2014-02-02 Thread Aaron Davidson
Sorry, I meant to say we will use the maximum between (the total number of cores in the cluster) and (2) if spark.default.parallelism is not set. So this should not be causing your problem unless your cluster thinks it has less than 2 cores. On Sun, Feb 2, 2014 at 4:46 PM, Aaron Davidson wrote

Re: default parallelism in trunk

2014-02-02 Thread Aaron Davidson
Could you give an example where default parallelism is set to 2 where it didn't used to be? Here is the relevant section for the spark standalone mode: CoarseGrainedSchedulerBackend.scala#L211

Re: Spark app gets slower as it gets executed more times

2014-02-02 Thread Aaron Davidson
Are you seeing any exceptions in between running apps? Does restarting the master/workers actually cause Spark to speed back up again? It's possible, for instance, that you run out of disk space, which should cause exceptions but not go away by restarting the master/workers. One thing to worry abo

Re: Why does sortByKey launch cluster job?

2014-01-08 Thread Aaron Davidson
Feel free to always file official bugs in Jira, as long as it's not already there! On Tue, Jan 7, 2014 at 9:47 PM, Andrew Ash wrote: > Hi Josh, > > I just ran into this again myself and noticed that the source hasn't > changed since we discussed in December. Should I file an official bug in >

Re: How to make Spark merge the output file?

2014-01-07 Thread Aaron Davidson
HDFS, since 0.21 , has a concat() method which would do exactly this, but I am not sure of the performance implications. Of course, as Matei pointed out, it's unusual to actually need a single HDFS file. On Mon, Jan 6, 2014 at 9:08 PM, Matei Zaharia

Re: How to access global kryo instance?

2014-01-06 Thread Aaron Davidson
; > > On Tue, Jan 7, 2014 at 2:52 AM, Aaron Davidson wrote: > >> Please take a look at the source code -- it's relatively friendly, and >> very useful for digging into Spark internals! >> (KryoSerializer<https://github.com/apache/incubator-spark/blob/master/

Re: How to access global kryo instance?

2014-01-06 Thread Aaron Davidson
ld use: > > val ser = SparkEnv.get.serializer.asInstanceOf[KryoSerializer] > > But how to get the instance of Kryo that spark uses from ser? > > > On Tue, Jan 7, 2014 at 1:04 AM, Aaron Davidson wrote: > >> I believe SparkEnv.get.serializer would return the serializer created >> from the

Re: How to access global kryo instance?

2014-01-06 Thread Aaron Davidson
I believe SparkEnv.get.serializer would return the serializer created from the "spark.serializer" property. You can also obtain a Kryo serializer directly via it's no-arg constructor (it still invokes your spark.kryo.registrator): val serializer = new KryoSerializer() but this could have some over

Re: ADD_JARS doesn't properly work for spark-shell

2014-01-04 Thread Aaron Davidson
formation. > > > > On Sun, Jan 5, 2014 at 4:06 AM, Aureliano Buendia wrote: > >> Sorry, I had a typo. I can conform that using ADD_JARS together with >> SPARK_CLASSPATH works as expected in spark-shell. >> >> It'd make sense to have the two combined

Re: State of spark on scala 2.10

2014-01-04 Thread Aaron Davidson
Scala 2.10.3 support was recently merged into master (#259). The branch is probably not as stable as 0.8.1, but things "should" work. The 2.10 branch should be deleted, the only issue is there are some outstanding PRs against that branch that hav

Re: ADD_JARS doesn't properly work for spark-shell

2014-01-04 Thread Aaron Davidson
014 at 7:07 PM, Aureliano Buendia wrote: > > > > On Sun, Jan 5, 2014 at 2:28 AM, Aaron Davidson wrote: > >> Additionally, which version of Spark are you running? >> > > 0.8.1. > > Unfortunately, this doesn't work either: > > MASTER=local[2] ADD_JARS

Re: ADD_JARS doesn't properly work for spark-shell

2014-01-04 Thread Aaron Davidson
Additionally, which version of Spark are you running? On Sat, Jan 4, 2014 at 6:27 PM, Aaron Davidson wrote: > I am not an expert on these classpath issues, but if you're using local > mode, you might also try to set SPARK_CLASSPATH to include the path to the > jar file as well. T

Re: ADD_JARS doesn't properly work for spark-shell

2014-01-04 Thread Aaron Davidson
I am not an expert on these classpath issues, but if you're using local mode, you might also try to set SPARK_CLASSPATH to include the path to the jar file as well. This should not really help, since "adding jars" is the right way to get the jars to your executors (which is where the exception appe

Re: Noob Spark questions

2013-12-30 Thread Aaron Davidson
Not sure if it helps, but there is a ZeroMQ Spark Streaming example: https://github.com/apache/incubator-spark/blob/master/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala . On Mon, Dec 30, 2013 at 9:41 PM, Ognen Duzlevski wrote: > Can anyone provide any code exa

Re: NPE while reading broadcast variable.

2013-12-30 Thread Aaron Davidson
Could you post the stack trace you see for the NPE? On Mon, Dec 30, 2013 at 11:31 AM, Archit Thakur wrote: > I am still getting it. I googled and found a similar open problem on > stackoverflow: > http://stackoverflow.com/questions/17794664/accumulator-fails-on-cluster-works-locally > . > > Thx,

Re: How to map each line to (line number, line)?

2013-12-30 Thread Aaron Davidson
> > It shouldn't be specific to text files, the same should happen with binary > files What is the expected notion of a line/line number in a binary file? On Mon, Dec 30, 2013 at 8:27 AM, Aureliano Buendia wrote: > > > > On Mon, Dec 30, 2013 at 4:24 PM, Michael (Bach) Bui wrote: > >> Note that

Re: How to set Akka frame size

2013-12-24 Thread Aaron Davidson
The error you're receiving is because the Akka frame size must be a positive Java Integer, i.e., less than 2^31. However, the frame size is not intended to be nearly the size of the job memory -- it is the smallest unit of data transfer that Spark does. In this case, your "task result" size is exce

Re: IMPORTANT: Spark mailing lists moving to Apache by September 1st

2013-12-19 Thread Aaron Davidson
I'd be fine with one-way mirrors here (Apache threads being reflected in Google groups) -- I have no idea how one is supposed to navigate the Apache list to look for historic threads. On Thu, Dec 19, 2013 at 7:58 PM, Mike Potts wrote: > Thanks very much for the prompt and comprehensive reply!

Re: problems with standalone cluster

2013-12-12 Thread Aaron Davidson
You might also check the spark/work/ directory for application (Executor) logs on the slaves. On Tue, Nov 19, 2013 at 6:13 PM, Umar Javed wrote: > I have a scala script that I'm trying to run on a Spark standalone cluster > with just one worker (existing on the master node). But the application

Re: groupBy() with really big groups fails

2013-12-09 Thread Aaron Davidson
GB to several > terabytes in size – although I'm not sure how big the groups themselves > will be (presumably proportional to the size of the dataset). > > Thanks for the quick response! > > -Matt Cheah > > From: Aaron Davidson > Reply-To: "user@spark.incuba

Re: groupBy() with really big groups fails

2013-12-09 Thread Aaron Davidson
This is very likely due to memory issues. The problem is that each "reducer" (partition of the groupBy) builds an in-memory table of that partition. If you have very few partitions, this will fail, so the solution is to simply increase the number of reducers. For example: sc.parallelize(1 to 40

Re: Serializable incompatible with Externalizable error

2013-12-03 Thread Aaron Davidson
This discussion seems to indicate the possibility of a mismatch between one side being Serializable and the other being Externalizable: https://forums.oracle.com/thread/2147644 In general, the semantics of Serializable can be pretty strange as it doesn't really behave the same as usual interfaces.

Re: Zookeeper and Spark

2013-12-03 Thread Aaron Davidson
I found this online, which may relate: http://stackoverflow.com/questions/13316776/zookeeper-connection-error Additionally, I'd check the zookeeper logs to see if they have any more information. On Tue, Dec 3, 2013 at 6:57 AM, Eduardo Costa Alfaia < e.costaalf...@studenti.unibs.it> wrote: > Hi D

Re: oome from blockmanager

2013-12-01 Thread Aaron Davidson
Thanks for the update, and glad to hear things are working well! On Sun, Dec 1, 2013 at 1:56 PM, Stephen Haberman wrote: > > > The short term solutions have already been discussed: decrease the > > number of reducers (and mappers, if you need them to be tied) or > > potentially turn off compres

Re: master and scala-2.10 merge

2013-12-01 Thread Aaron Davidson
CCing Matei and Prashant. I believe there are outstanding issues which are inhibiting the merge from the scala-2.10 merge into master. We've had such a merge PR ( https://github.com/apache/incubator-spark/pull/211) outstanding for a while now, it just hasn't been clear which way we should merge th

Re: spark-shell not working on standalone cluster (java.io.IOException: Cannot run program compute-classpath.sh)

2013-11-25 Thread Aaron Davidson
There is a pull request currently to fix this exact issue, I believe, at https://github.com/apache/incubator-spark/pull/192. It's very small and only touches the script files, so you could apply it to your current version and distribute it to the workers. The fix here is that you add an additional

Re: oome from blockmanager

2013-11-22 Thread Aaron Davidson
u're using compression or seeing more than 48k DiskBlockObjectWriters to account for the remaining memory used. On Fri, Nov 22, 2013 at 9:05 AM, Aaron Davidson wrote: > Great, thanks for the feedback. It sounds like you're using the LZF > compression scheme -- switching to Snappy sho

Re: oome from blockmanager

2013-11-22 Thread Aaron Davidson
sily got OOM. > > > > What a dilemma is that my application needs groupByKey transformation > which requires small partitions size, but small partition size will lead to > more partition numbers that also consumes lots of memory. > > > > Thanks > > Jerry > &g

Re: oome from blockmanager

2013-11-21 Thread Aaron Davidson
Thanks for your feedback; I think this is a very important issue on the usability front. One thing to consider is that at some data size, one simply needs larger or more nodes. m1.large is essentially the smallest ec2 instance size that can run a Spark job of any reasonable size. That's not an excu

Re: oome from blockmanager

2013-11-21 Thread Aaron Davidson
There is distinction to be made between the number of incoming partitions and the number of reducers. Let's say that the number of partitions is more or less irrelevant, especially since we don't directly control the number of partitions in the input data set (as you pointed out). More significant

Re: EC2 node submit jobs to separate Spark Cluster

2013-11-19 Thread Aaron Davidson
driver? I've heard this is required. And > obviously allowing all traffic is bad… > > -Matt Cheah > > From: Aaron Davidson > Reply-To: "user@spark.incubator.apache.org" < > user@spark.incubator.apache.org> > Date: Monday, November 18, 2013 8:28 PM

Re: Can not get the expected output when running the BroadcastTest example program.

2013-11-18 Thread Aaron Davidson
Assuming your cluster is actually working (e.g., other examples like SparkPi work), then the problem is probably that println() doesn't actually write output back to the driver; instead, it may just be outputting locally to each slave. You can test this by replacing lines 43 through 45 with: sc.

Re: EC2 node submit jobs to separate Spark Cluster

2013-11-18 Thread Aaron Davidson
The main issue with running a spark-shell locally is that it orchestrates the actual computation, so you want it to be "close" to the actual Worker nodes for latency reasons. Running a spark-shell on EC2 in the same region as the Spark cluster avoids this problem. The error you're seeing seems to

Re: debugging a Spark error

2013-11-18 Thread Aaron Davidson
Have you looked a the Spark executor logs? They're usually located in the $SPARK_HOME/work/ directory. If you're running in a cluster, they'll be on the individual slave nodes. These should hopefully reveal more information. On Mon, Nov 18, 2013 at 3:42 PM, Chris Grier wrote: > Hi, > > I'm tryin

Re: failure notice

2013-11-17 Thread Aaron Davidson
~/Downloads/spark-0.8.0-incubating $ ant -v > Apache Ant(TM) version 1.8.2 compiled on May 18 2012 > Mvh/BR > Egon Kidmose > > > On Sun, Nov 17, 2013 at 7:40 PM, Aaron Davidson > wrote: > > Could you report your ant/Ivy version? Just run > > ant -version > > >

Re: failure notice

2013-11-17 Thread Aaron Davidson
Could you report your ant/Ivy version? Just run ant -version The fundamental problem is that Ivy is stupidly thinking ".orbit" is the file extension when it should be ".jar". There are two possible fixes you can try, and please let us know if one or the other works. In $SPARK_HOME/project/SparkBui

Re: number of splits for standalone cluster mode

2013-11-17 Thread Aaron Davidson
The number of splits can be configured when reading the file, as an argument to textFile(), sequenceFile(), etc (see docs). Note that this is a minimum, however, as cert

Re: foreachPartition in Java

2013-11-17 Thread Aaron Davidson
Also, in general, you can workaround shortcomings in the Java API by converting to a Scala RDD (using JavaRDD's rdd() method). The API tends to be much clunkier since you have to jump through some hoops to talk to a Scala API in Java, though. In this case, JavaRDD's mapPartition() method will likel

Re: How to add more worker node to spark cluster on EC2

2013-11-17 Thread Aaron Davidson
Hi Xiaobing, At its heart, this is a very easy thing to do. Instead of the master reaching out to the workers, the worker just needs to find the master. In standalone mode, this can be accomplished simply by setting the SPARK_MASTER_IP/_PORT variables in spark-env.sh. In order to make the other s

Re: Memory configuration in local mode

2013-11-16 Thread Aaron Davidson
e using local mode, you can just pass -Xmx32g to the JVM that > is launching spark and it will have that much memory. > > On Fri, Nov 15, 2013 at 6:30 PM, Aaron Davidson > wrote: > > One possible workaround would be to use the local-cluster Spark mode. > This > > is nor

Re: Memory configuration in local mode

2013-11-15 Thread Aaron Davidson
One possible workaround would be to use the local-cluster Spark mode. This is normally used only for testing, but it will actually spawn a separate process for the executor. The format is: new SparkContext("local-cluster[1,4,32000]") This will spawn 1 Executor that is allocated 4 cores and 32GB (ap

Re: code review - splitting columns

2013-11-15 Thread Aaron Davidson
Regarding only your last point, you could always split backwards to avoid having to worry about updated indices (i.e., split the highest index column first). But if you're additionally worried about efficiency, a combined approach could make more sense to avoid making two full passes on the data.

Re: any java api to get spark cluster info

2013-11-14 Thread Aaron Davidson
Hi! I think I have the maximally horrendous solution to this problem. If you just want to know the total cores of a Standalone or Coarse Grained scheduler, and are OK with going "off trail" of the public API, so to speak, you can use something like the following (just beware that it's liable to bre

Re: mapping of shuffle outputs to reduce tasks

2013-11-10 Thread Aaron Davidson
It is responsible for a subset of shuffle blocks. MapTasks split up their data, creating one shuffle block for every reducer. During the shuffle phase, the reducer will fetch all shuffle blocks that were intended for it. On Sun, Nov 10, 2013 at 9:38 PM, Umar Javed wrote: > I was wondering how d

Re: oome from blockmanager

2013-11-05 Thread Aaron Davidson
It is available in branch-0.8 and master. On Tue, Nov 5, 2013 at 10:23 AM, Ashish Rangole wrote: > Awesome indeed! Is this available in the main branch now? > > Thanks! > On Nov 5, 2013 9:50 AM, "Stephen Haberman" > wrote: > >> >> > As a followup on this, the memory footprint of all shuffle me

Re: oome from blockmanager

2013-11-05 Thread Aaron Davidson
As a followup on this, the memory footprint of all shuffle metadata has been greatly reduced. For your original workload with 7k mappers, 7k reducers, and 5 machines, the total metadata size should have decreased from ~3.3 GB to ~80 MB. On Tue, Oct 29, 2013 at 9:07 AM, Aaron Davidson wrote

Re: Executor could not connect to Driver?

2013-11-01 Thread Aaron Davidson
I've seen this happen before due to the driver doing long GCs when the driver machine was heavily memory-constrained. For this particular issue, simply freeing up memory used by other applications fixed the problem. On Fri, Nov 1, 2013 at 12:14 AM, Liu, Raymond wrote: > Hi > > I am encounter an

Re: repartitioning RDDS

2013-10-31 Thread Aaron Davidson
Stephen is exactly correct, I just wanted to point out that in Spark 0.8.1 and above, the "repartition" function has been added to be a clearer way to accomplish what you want. ("Coalescing" into a larger number of partitions doesn't make much linguistic sense.) On Thu, Oct 31, 2013 at 7:48 AM, S

Re: Spark cluster memory configuration for spark-shell

2013-10-29 Thread Aaron Davidson
You are correct. If you are just using spark-shell in local mode (i.e., without cluster), you can set the SPARK_MEM environment variable to give the driver more memory. E.g.: SPARK_MEM=24g ./spark-shell Otherwise, if you're using a real cluster, the driver shouldn't require a significant amount of

Re: oome from blockmanager

2013-10-29 Thread Aaron Davidson
Great! Glad to hear it worked out. Spark definitely has a pain point about deciding the right number of partitions, and I think we're going to be spending a lot of time trying to reduce that issue. Currently working on the patch to reduce the shuffle file block overheads, but in the meantime, you

Re: oome from blockmanager

2013-10-26 Thread Aaron Davidson
You're precisely correct about your (N*M)/(# machines) shuffle blocks per machine. I believe the 5.5 million data structures instead of 9.8 comes from the fact that the shuffle was only around 50% of the way through before it blew up. Technically, the ShuffleMapTask should not require buffering th

Re: oome from blockmanager

2013-10-26 Thread Aaron Davidson
Thanks again for the great detail! Your setup sounds correct, and all the numbers you're seeing suggest that no foul play (i.e., bug) is at work. I think this is a simple case of the blockInfo and blockToFileSegmentMap being poorly optimized, especially the latter. I will look into reducing the mem

Re: almost sorted data

2013-10-25 Thread Aaron Davidson
Currently, our sortByKey should be using Java's native Timsort implementation, which is an adaptive sort. That should also mean sorting is very fast for almost-sorted data. The overhead you're seeing might be caused by reshuffling everything during the range partitioning step *before *the sort, whi

Re: gc/oome from 14,000 DiskBlockObjectWriters

2013-10-25 Thread Aaron Davidson
Snappy sounds like it'd be a better solution here. LZF requires a pretty sizeable buffer per stream (accounting for the 300k you're seeing). It looks like you have 7000 reducers, and each one requires an LZF-compressed stream. Snappy has a much lower overhead per stream, so I'd give it a try. Than

Re: Visitor function to RDD elements

2013-10-22 Thread Aaron Davidson
I may be incorrect on this, but I believe that coalesce() could work here. A single partition can be larger than memory, as long as no one tries to load the entire partition into memory. Trouble should only arise if your RDD operator tries to create an in-memory map of the entire partition (e.g., r

Re: Spark Streaming - How to control the parallelism like storm

2013-10-22 Thread Aaron Davidson
As Mark said, flatMap can only parallelize into as many partitions as exist in the incoming RDD. socketTextStream() only produces 1 RDD at a time. However, you can utilize the RDD.coalesce() method to split one RDD into multiple (excuse the name; it can be used for shrinking or growing the number o

Re: Help with Initial Cluster Configuration / Tuning

2013-10-22 Thread Aaron Davidson
On the other hand, I totally agree that memory usage in Spark is rather opaque, and is one area where we could do a lot better at in terms of communicating issues, through both docs and instrumentation. At least with serialization and such, you can get meaningful exceptions (hopefully), but OOMs ar

Re: Broken link in quickstart

2013-10-22 Thread Aaron Davidson
Thanks for the heads up! I have submitted a pull request to fix it (#98), so it should be corrected soon. In the meantime, if anyone is curious, the real link should be http://spark.incubator.apache.org/docs/latest/api/core/index.html#org.apache.sp

Re: Spark unit test question

2013-10-21 Thread Aaron Davidson
To answer your second question first, you can use the SparkContext format "local-cluster[2, 1, 512]" (instead of "local[2]"), which would create a local test cluster with 2 workers, each with 1 core and 512 MB of memory. This should allow you to accurately test things like serialization. I don't b

Re: RDD sample fraction precision

2013-10-21 Thread Aaron Davidson
Perhaps I'm misunderstanding your question, but RDD.sample() just uses the fraction as the probability of accepting a given tuple (rather than, say, taking every 7th tuple). So on average, 1/7 of the tuples will be returned. For small input sizes, though, this could return significantly more or les

Re: job reports as KILLED in standalone mode

2013-10-18 Thread Aaron Davidson
Whenever an Executor ends, it enters into one of three states: KILLED, FAILED, LOST (see: 1). None of these sound like "exited cleanly," which I a

Re: Spark REPL produces error on a piece of scala code that works in pure Scala REPL

2013-10-12 Thread Aaron Davidson
the Scala shell: > > > https://groups.google.com/forum/#!searchin/spark-users/error$3A$20type$20mismatch|sort:relevance/spark-users/bwAmbUgxWrA/HwP4Nv4adfEJ > > > On Fri, Oct 11, 2013 at 6:01 PM, Aaron Davidson wrote: > >> Playing around with this a little more, it

Re: Spark REPL produces error on a piece of scala code that works in pure Scala REPL

2013-10-11 Thread Aaron Davidson
Playing around with this a little more, it seems that classOf[Animal] is "this.Animal" in Spark and "Animal" in normal Scala. Also, trying to do something like this: class Zoo[A <: *this.*Animal](thing: A) { } works in Scala but throws a weird error in Spark: "error: type Animal is not a member o

Re: spark_ec2 script in 0.8.0 and mesos

2013-10-08 Thread Aaron Davidson
Also, please post feature requests here: http://spark-project.atlassian.net Make sure to search prior to posting to avoid duplicates. On Tue, Oct 8, 2013 at 11:50 AM, Matei Zaharia wrote: > Hi Shay, > > We actually don't support Mesos in the EC2 scripts anymore -- sorry about > that. If you want

Re: spark through vpn, SPARK_LOCAL_IP

2013-10-05 Thread Aaron Davidson
You might try also setting spark.driver.host to the correct IP in the conf/spark-env.sh SPARK_JAVA_OPTs as well. e.g., -Dspark.driver.host=192.168.250.47 On Sat, Oct 5, 2013 at 2:45 PM, Aaron Babcock wrote: > Hello, > > I am using spark through a vpn. My driver machine ends up with two ip > ad