Re: Spark process locality

2014-02-19 Thread Patrick Wendell
I think these are fairly well explained in the user docs. Was there something unclear that maybe we could update? http://spark.incubator.apache.org/docs/latest/configuration.html On Wed, Feb 19, 2014 at 10:04 AM, Mayur Rustagi mayur.rust...@gmail.com wrote: Process local implies the data is

Re: Unable to submit an application to standalone cluster which on hdfs.

2014-02-19 Thread Patrick Wendell
Thanks for reporting this - this is a bug with the way it validates the URL. I'm filing this as a blocker for 0.9.1. If you are able to compile Spark, try just removing the validation block. On Tue, Feb 18, 2014 at 10:27 PM, samuel281 samuel...@gmail.com wrote: Actually I tried them both.

Re: working on a closed networkworking on a closed network - any recomendations

2014-02-09 Thread Patrick Wendell
In general it's hard to do builds on machines that don't access the internet. The most common thing in environments where building is happening regularly is to set up and internal repositories (such as a nexus repository) where dependencies can be added to that repository manually. If it's just a

Re: spark 0.9.0 compatible with hadoop 1.0.4 ?

2014-02-06 Thread Patrick Wendell
It looks like you have multiple copies of the assembly jar (see the error at the top). Try doing a clean before you build sbt/sbt clean assembly On Thu, Feb 6, 2014 at 6:20 PM, Suhas Satish suhas.sat...@gmail.com wrote: Is spark compatible with Hadoop 1.0.4 ? I built spark with the variable -

Re: Announcing Calliope releases 0.8.1-GA and 0.9.0-EA

2014-02-04 Thread Patrick Wendell
This is cool Rohit, thanks for announcing. On Tue, Feb 4, 2014 at 6:37 AM, Sam Bessalah sam.bessa...@gmail.com wrote: Congrats. Great job. I really like what you are doing with Calliope. On Tue, Feb 4, 2014 at 3:01 PM, Rohit Rai ro...@tuplejump.com wrote: Hi All, We are glad to announce

Re: ExternalAppendOnlyMap throw no such element

2014-01-26 Thread Patrick Wendell
out the cause. Thanks, Jiacheng Guo On Wed, Jan 22, 2014 at 1:36 PM, Patrick Wendell pwend...@gmail.com wrote: This code has been modified since you reported this so you may want to try the current master. - Patrick On Mon, Jan 20, 2014 at 4:22 AM, guojc guoj...@gmail.com wrote: Hi

Re: Using persistent hdfs on spark ec2 instanes

2014-01-22 Thread Patrick Wendell
1. It seems by default spark ec2 uses ephemeral hdfs, how to switch this to persistent hdfs? You can stop the ephemeral one using /root/ephemeral-hdfs/bin/stop-dfs.sh and start the persistent one using /root/persistent-hdfs/bin/start-dfs.sh 2. By default persistent hdfs server is not up,

Re: Using persistent hdfs on spark ec2 instanes

2014-01-22 Thread Patrick Wendell
1. It seems by default spark ec2 uses ephemeral hdfs, how to switch this to persistent hdfs? You can stop the ephemeral one using /root/ephemeral-hdfs/bin/stop-dfs.sh and start the persistent one using /root/persistent-hdfs/bin/start-dfs.sh 2. By default persistent hdfs server is not up,

Re: Using persistent hdfs on spark ec2 instanes

2014-01-22 Thread Patrick Wendell
1. It seems by default spark ec2 uses ephemeral hdfs, how to switch this to persistent hdfs? You can stop the ephemeral one using /root/ephemeral-hdfs/bin/stop-dfs.sh and start the persistent one using /root/persistent-hdfs/bin/start-dfs.sh 2. By default persistent hdfs server is not up,

Re: Using persistent hdfs on spark ec2 instanes

2014-01-22 Thread Patrick Wendell
is set to 9010 port, instead of 9000. Does spark need more config for this? On Thu, Jan 23, 2014 at 12:26 AM, Patrick Wendell pwend...@gmail.com wrote: 1. It seems by default spark ec2 uses ephemeral hdfs, how to switch this to persistent hdfs? You can stop the ephemeral one using

Re: Spark does not retry failed tasks initiated by hadoop

2014-01-22 Thread Patrick Wendell
What makes you think it isn't retrying the task? By default it tries three times... it only prints the error once though. in this case if your cluster doesn't have any datanodes it's likely that it failed several times. On Wed, Jan 22, 2014 at 4:04 PM, Aureliano Buendia buendia...@gmail.com

Re: SparkException: Expect only DirectTaskResults when using localScheduler()

2014-01-18 Thread Patrick Wendell
at 9:54 AM, Patrick Wendell pwend...@gmail.com wrote: This is a bug that was fixed and will be part of 0.8.2: https://github.com/apache/incubator-spark/pull/281 A workaround is setting the akka frame size to be larger using spark.akka.frameSize. The issue is that we added a mechanism

Re: Worker hangs with 100% CPU in Standalone cluster

2014-01-16 Thread Patrick Wendell
Thanks for following up and explaining this one! Definitely something other users might run into... On Thu, Jan 16, 2014 at 5:58 AM, Grega Kešpret gr...@celtra.com wrote: Just to follow up, we have since pinpointed the problem to be in application code (not Spark). In some cases, there was an

Re: Akka error kills workers in standalone mode

2014-01-16 Thread Patrick Wendell
We included a patch in 0.9.0-rc1 (currently being voted) that uses a default SPARK_HOME if the user doesn't specify it. Having an NPE is indeed bad behavior here. Thanks for reporting this. - Patrick On Thu, Jan 16, 2014 at 4:09 AM, vuakko niko.vuo...@gmail.com wrote: Yes, thanks for the help.

Re: Consistency between RDD's and Native File System

2014-01-16 Thread Patrick Wendell
RDD's are immutable, so there isn't really such a thing as modifying a block in-place inside of an RDD. As a result, this particular consistency issue doesn't come up in Spark. - Patrick On Thu, Jan 16, 2014 at 1:42 AM, SaiPrasanna sai.annama...@siemens.com wrote: Hello, i am a novice to SPARK

Re: How does shuffle work in spark ?

2014-01-16 Thread Patrick Wendell
The intermediate shuffle output gets written to disk, but it often hits the OS-buffer cache since it's not explicitly fsync'ed, so in many cases it stays entirely in memory. The behavior of the shuffle is agnostic to whether the base RDD is in cache or in disk. For on-disk RDD's or inputs, the

Re: Problem running example GroupByTest from scala command line

2014-01-12 Thread Patrick Wendell
You should launch with java and not scala to launch. The scala command in newer versions manually adds a specific version of akka to the classpath which conflicts with the version spark is using. This causes the error you are seeing. It's discussed in this thread on the dev list:

Re: Development version error on sbt compile publish-local

2014-01-12 Thread Patrick Wendell
On Sunday, January 12, 2014 12:32 AM, Patrick Wendell pwend...@gmail.com wrote: Can you try running sbt/sbt clean. Sometimes things can get randomly corrupted and cause stuff like this. On Sat, Jan 11, 2014 at 12:49 PM, Shing Hing Man mat...@yahoo.com wrote: Hi, I have checkouted

Re: Development version error on sbt compile publish-local

2014-01-11 Thread Patrick Wendell
Can you try running sbt/sbt clean. Sometimes things can get randomly corrupted and cause stuff like this. On Sat, Jan 11, 2014 at 12:49 PM, Shing Hing Man mat...@yahoo.com wrote: Hi, I have checkouted the development version of Spark at

Re: Troubles with the Spark-EC2 stuff

2014-01-04 Thread Patrick Wendell
Look in /root/mapreduce. This is different for hadoop2 clusters because mapreduce is now distributed as a separate project. On Sat, Jan 4, 2014 at 2:04 PM, Guillaume Pitel guillaume.pi...@exensa.comwrote: Hi, Thanks, it wasn't actually the problem but your suggestion made me found it. I've

Re: State of spark on scala 2.10

2014-01-04 Thread Patrick Wendell
to a directory, or does it need a lot of configurations? On Sun, Jan 5, 2014 at 5:03 AM, Patrick Wendell pwend...@gmail.com wrote: I meant you'll need to build your own version of Spark. Typically we do this by launching an existing AMI and then just building a new version of spark and copying

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

2013-12-24 Thread Patrick Wendell
Hey Andy - these Nabble groups look great! Thanks for setting them up. On Tue, Dec 24, 2013 at 10:49 AM, Evan Chan e...@ooyala.com wrote: Thanks Andy, at first glance nabble seems great, it allows search plus posting new topics, so it appears to be bidirectional.Now just have to register

Re: failed to compile spark because of the missing packages

2013-12-23 Thread Patrick Wendell
Hey Nan, You shouldn't copy lib_managed manually. SBT will deal with that. Try just using the same .gitignore settings that we have in the spark github. Seems like you are accidentally including some files that cause this to get messed up. - Patrick On Mon, Dec 23, 2013 at 8:37 AM, Nan Zhu

Re: Spark shell vs Spark job

2013-12-19 Thread Patrick Wendell
You can use the UI to debug what is going on. For instance, are the tasks themselves taking longer (?) or is it possible that the overall job acquires fewer executors (?). Another thing influencing this is that when you run a job it counts the time it takes to go and start-up all the executors.

Re: Spark streaming vs. spark usage

2013-12-18 Thread Patrick Wendell
Hey Nathan, If I understand correctly, that calls existingBatchStuff on each RDD from which the window is made, not on the window as a whole. If I want a combined result across the whole window, I'm not sure how to do that directly. Actually, once you call window() you get a new sequence

Re: Repartitioning an RDD

2013-12-17 Thread Patrick Wendell
Master and 0.8.1 (soon to be released) have `repartition`. It's actually a new feature not an old one! On Tue, Dec 17, 2013 at 4:31 PM, Mark Hamstra m...@clearstorydata.com wrote: https://github.com/apache/incubator-spark/blob/master/core/src/main/scala/org/apache/spark/rdd/RDD.scala#L280 On

Re: spark pre-built binaries for 0.8.0

2013-12-17 Thread Patrick Wendell
Hey Philip, No - those are compiled against the mr1 version. You'll need to build yourself for YARN. - Patrick On Tue, Dec 17, 2013 at 10:32 AM, Philip Ogren philip.og...@oracle.com wrote: I have a question about the pre-built binary for 0.8.0 for CDH 4 listed here:

Re: exposing spark through a web service

2013-12-13 Thread Patrick Wendell
Hey Philip, To elaborate a bit, this is a proposed patch for integrating something like a restful server into Spark. If you wanted to take a look at the documentation in that patch and comment as to whether it would partially or fully solve your use-case that would be great. - Patrick On Fri,

Re: some wrong link in Spark Summit web page

2013-12-13 Thread Patrick Wendell
Thanks for reporting this we'll figure it out. On Fri, Dec 13, 2013 at 10:04 AM, Nan Zhu zhunanmcg...@gmail.com wrote: Hi, I'm not sure if it is the right place to talk about this, if not, I'm very sorry about that - 9-9:30am The State of Spark, and Where We’re Going

Re: spark 0.8.0 fails on larger data set (Failed to run reduce at GradientDescent.scala:144)

2013-12-12 Thread Patrick Wendell
See if there are any logs on the slaves that suggest why the tasks are failing. Right now the master log is just saying some stuff is failing but it's not clear why. On Thu, Dec 12, 2013 at 9:36 AM, Taka Shinagawa taka.epsi...@gmail.comwrote: How big is your data set? Did you set SPARK_MEM

Re: Bump: on disk storage formats

2013-12-08 Thread Patrick Wendell
(although i don't have any experience with parquet files). What is the recommended way of interacting(read/write) with parquet files? -- Ankur On 8 Dec 2013, at 17:38, Patrick Wendell pwend...@gmail.com wrote: This is a very open ended question so it's hard to give a specific answer

Re: Spark Error Log

2013-11-28 Thread Patrick Wendell
Hey Wenlei, There is some issue in master that is repressing the log output - I'm trying to debug it before we release 0.8.1. Can you explain exactly how you are running Spark? Are you running the shell or are you running a standalone application? - Patrick On Thu, Nov 28, 2013 at 12:54 AM,

Re: Using Cassandra as an input stream from Java

2013-11-28 Thread Patrick Wendell
Tal - that would be great to have open sourced if you can do it! On Thu, Nov 28, 2013 at 1:00 AM, Pulasthi Supun Wickramasinghe pulasthi...@gmail.com wrote: Hi Tal, Thanks for the info will try it out and see how it goes. On Thu, Nov 28, 2013 at 2:19 PM, Tal Sliwowicz ta...@taboola.com

Re: local[k] job gets stuck - spark 0.8.0

2013-11-28 Thread Patrick Wendell
gets stuck. Thx Vijay Gaikwad University of Washington MSIM vijay...@gmail.com (206) 261-5828 On Nov 27, 2013, at 1:34 PM, Patrick Wendell pwend...@gmail.com wrote: Vijay - you said the job gets stuck but you also said it eventually completes. What do you mean by stuck? Do you mean

Re: Spark Error Log

2013-11-28 Thread Patrick Wendell
Sharma scrapco...@gmail.com wrote: I think all that is needed is an log4j.properties on the classpath http://logging.apache.org/log4j/1.2/faq.html#noconfig On Thu, Nov 28, 2013 at 11:52 PM, Patrick Wendell pwend...@gmail.com wrote: Hey Wenlei, There is some issue in master that is repressing

Re: Setting spark.akka.frameSize to the max

2013-11-26 Thread Patrick Wendell
results specifically or some other stage. I'll try to get you a copy of that stacktrace so we have something tangible to discuss. Andrew On Mon, Nov 25, 2013 at 10:10 AM, Patrick Wendell pwend...@gmail.com wrote: Good question, I think inside of akka they will allocate a buffer

Re: Spark, EC2, and CDH4 Questions

2013-11-26 Thread Patrick Wendell
Is it possible for the spark EC2 scripts to deploy clusters set up with Cloudera's CDH4 hadoop distribution, as opposed to the default hadoop distributions? No this isn't possible. The EC2 scripts are designed to launch a cluster from scratch with a specific configuration. However, if you've

Re: local[k] job gets stuck - spark 0.8.0

2013-11-25 Thread Patrick Wendell
When it gets stuck, what does it show in the web UI? Also, can you run a jstack on the process and attach the output... that might explain what's going on. On Mon, Nov 25, 2013 at 11:30 AM, Vijay Gaikwad vijay...@gmail.com wrote: I am using apache spark 0.8.0 to process a large data file and

Re: Spark driver behind NAT

2013-11-24 Thread Patrick Wendell
Currently, no. The driver is a stateful component that is heavy-weight and should be run inside of the cluster. On Fri, Nov 22, 2013 at 4:49 PM, Dmitriy Lyubimov dlie...@gmail.com wrote: Hello, as far as i can tell, spark executors use akka to connect back to the driver. However, if driver

Re: Spark driver behind NAT

2013-11-24 Thread Patrick Wendell
Or more generally, it should be fully visible to the slave nodes inside of the cluster. On Sun, Nov 24, 2013 at 3:16 PM, Patrick Wendell pwend...@gmail.com wrote: Currently, no. The driver is a stateful component that is heavy-weight and should be run inside of the cluster. On Fri, Nov 22

Re: Problem with Multi-user In Spark

2013-11-22 Thread Patrick Wendell
Spark has good support for multi-tenancy. Prabeesh could you describe in more detail what you are actually running, what the expected behavior is and what you are observing? On Fri, Nov 22, 2013 at 3:09 AM, Sam Bessalah samkil...@gmail.com wrote: Spark doesn't support multi users, as far as I

Re: problems with launching executor in standalone cluster

2013-11-22 Thread Patrick Wendell
It looks like scala is not on the classpath. Make sure SCALA_HOME is set correctly in spark-env.sh On Thu, Nov 21, 2013 at 6:54 PM, Umar Javed umarj.ja...@gmail.com wrote: Sorry if this seems abusing this list, but any idea what may be going on here? thanks! On Thu, Nov 21, 2013 at 12:55

Re: Opinions stratosphere

2013-11-22 Thread Patrick Wendell
I've never seen that project before, would be interesting to get a comparison. Seems to offer a much lower level API. For instance this is a wordcount program:

Re: setting SPARK_HADOOP_VERSION

2013-11-21 Thread Patrick Wendell
It was 4.2.0 - we've kept it this way for the last few releases for consistency. On Thu, Nov 21, 2013 at 11:19 AM, Pierre Borckmans pierre.borckm...@realimpactanalytics.com wrote: You are right about the typo. I did a copy paste from the website which presents this mistake. I forgot about

Re: Spark Configuration with Python

2013-11-20 Thread Patrick Wendell
You can add java options in SPARK_JAVA_OPTS inside of conf/spark-env.sh http://spark.incubator.apache.org/docs/latest/python-programming-guide.html#installing-and-configuring-pyspark - Patrick On Wed, Nov 20, 2013 at 8:52 AM, Michal Romaniuk michal.romaniu...@imperial.ac.uk wrote: The info

Re: Memory configuration in local mode

2013-11-16 Thread Patrick Wendell
If you are 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 ilike...@gmail.com wrote: One possible workaround would be to use the local-cluster Spark mode. This is normally used only

Re: Memory configuration in local mode

2013-11-16 Thread Patrick Wendell
...@gmail.com wrote: I was under the impression that he was using the same JVM for Spark and other stuff, and wanted to limit how much of it Spark could use. Patrick's solution is of course the right way to go if that's not the case. On Sat, Nov 16, 2013 at 9:40 AM, Patrick Wendell pwend

Re: code review - counting populated columns

2013-11-10 Thread Patrick Wendell
. Write-once code like the following is why I never liked Perl def cpc(tsv: RDD[String]) : RDD[(Int, Int)] = { tsv.flatMap(_.split(\t).zipWithIndex).filter(ci = ci._1.length0).map(ci = (ci._2, 1)).reduceByKey(_+_) } Thanks, Philip On 11/8/2013 2:41 PM, Patrick Wendell wrote

Re: code review - counting populated columns

2013-11-08 Thread Patrick Wendell
).map(line = line.split(\t).zipWithIndex.filter(_._1.length0)) sparseRows.reduce(mergeAdd(_,_)) You'll have to write a mergeAdd function. This might not be any faster, but it does allow variable length rows. On Fri, Nov 8, 2013 at 3:23 PM, Patrick Wendell pwend...@gmail.com wrote

Re: java.io.NotSerializableException on RDD count() in Java

2013-11-07 Thread Patrick Wendell
as transient. On Sun, Nov 3, 2013 at 7:32 PM, Patrick Wendell pwend...@gmail.com wrote: Hm, I think you are triggering a bug in the Java API where closures may not be properly cleaned. I think @rxin has reproduced this, deferring to him. - Patrick On Sun, Nov 3, 2013 at 5:25 PM, Yadid

Re: java.io.NotSerializableException on RDD count() in Java

2013-11-03 Thread Patrick Wendell
) at java.lang.Thread.run(Thread.java:695) I can post my code if that helps On 11/3/13 8:05 PM, Patrick Wendell wrote: If you look in the UI, are there failures on any of the slaves that you can give a stack trace for? That would narrow down where the serialization error is happening. Unfortunately

Re: Save RDDs as CSV

2013-10-30 Thread Patrick Wendell
You can do this if you coalesce the data first. However, this will put all of your final data through a single reduce tasks (so you get no parallelism and may overload a node): myrdd.coalesce(1).saveAsTextFile(hdfs:///my.csv) Basically you have to chose, either you do the write in parallel

Re: met a problem while running a streaming example program

2013-10-29 Thread Patrick Wendell
If you just add the extends Serializable changes from here it should work. On Tue, Oct 29, 2013 at 9:36 AM, Patrick Wendell pwend...@gmail.com wrote: This was fixed on 0.8 branch and master: https://github.com/apache/incubator-spark/pull/63/files - Patrick On Tue, Oct 29, 2013 at 9:17 AM

Re: Job duration

2013-10-28 Thread Patrick Wendell
...@readstraightup.com (201) 286-7785 On Mon, Oct 28, 2013 at 9:24 AM, Patrick Wendell pwend...@gmail.comwrote: Hey Lucas, Could you provide some rough psuedo-code for your job? One question is: are you loading the data from cassandra every time you perform an action, or do you cache

Re: Job duration

2013-10-28 Thread Patrick Wendell
: Finished TID 6782 in 7985 ms on domU-12-31-39-0A-90-F2.compute-1.internal (progress: 2221/2300) Thank you very much! Lucas. 2013/10/28 Patrick Wendell pwend...@gmail.com Hey Lucas, This code still needs to read the entire initial dataset from Cassandra, so that's probably what's taking

Re: Dependency while creating jar duplicate file.

2013-10-27 Thread Patrick Wendell
launching it. - Patrick On Sun, Oct 27, 2013 at 1:32 PM, Patrick Wendell pwend...@gmail.com wrote: Forgot to include user list... On Sun, Oct 27, 2013 at 1:16 PM, Patrick Wendell pwend...@gmail.com wrote: When you are creating an assembly jar you need to deal with all merge conflicts

Re: oome from blockmanager

2013-10-26 Thread Patrick Wendell
Hey Stephen, The issue is this. When you do a shuffle form N map partitions to M reduce partitions, there are N * M output blocks created and each one is tracked. That's why all these per-block overheads are causing you to OOM. One way to fix this is to reduce the per-block overhead which Josh

Re: oome from blockmanager

2013-10-26 Thread Patrick Wendell
may not need so many partitions. On Sat, Oct 26, 2013 at 2:33 PM, Patrick Wendell pwend...@gmail.com wrote: Hey Stephen, The issue is this. When you do a shuffle form N map partitions to M reduce partitions, there are N * M output blocks created and each one is tracked. That's why all

Re: Spark integration with HDFS and Cassandra simultaneously

2013-10-26 Thread Patrick Wendell
Hey Rohit, A single SparkContext can be used to read and write files of different formats, including HDFS or cassandra. For instance you could do this: rdd1 = sc.textFile(XXX) // Some text file in HDFS rdd1.saveAsHadoopFile(.., classOf[ColumnFamilyOutputFormat], ...) // Save into a cassandra

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

2013-10-24 Thread Patrick Wendell
This will be in 0.8.1 and 0.9.0 as the repartition() function. On Tue, Oct 22, 2013 at 11:25 PM, Patrick Wendell pwend...@gmail.comwrote: This is something we should add directly to the streaming API rather than requiring a transform() call. In fact, it's one of the really nice things about

Re: Loading leveldb into Spark..

2013-10-23 Thread Patrick Wendell
You could try reading it directly in Spark via JDBC, but for this size data you may want to do a higher throughput transfer into HDFS first and then load it from Spark. On Wed, Oct 23, 2013 at 10:36 PM, Himanshu Bafna (HB) hba...@yahoo.comwrote: Hi I have use case where in our data is stored

Re: Loading leveldb into Spark..

2013-10-23 Thread Patrick Wendell
load it from HDFS... On Wed, Oct 23, 2013 at 10:40 PM, Patrick Wendell pwend...@gmail.comwrote: You could try reading it directly in Spark via JDBC, but for this size data you may want to do a higher throughput transfer into HDFS first and then load it from Spark. On Wed, Oct 23, 2013

Re: Visitor function to RDD elements

2013-10-22 Thread Patrick Wendell
Hey Matt, It seems like you are trying to perform an operation that just isn't parrallelizable. In that case, it's going to be tricky without collecting the entire dataset on one node. Spark does not expose an iterator like you are suggesting, that lets you traverse an RDD. You could build one