Re: run spark0.9.1 on yarn with hadoop CDH4

2014-05-16 Thread Sandy Ryza
Hi Sophia, Unfortunately, Spark doesn't work against YARN in CDH4. The YARN APIs changed quite a bit before finally being stabilized in Hadoop 2.2 and CDH5. Spark on YARN supports Hadoop 0.23.* and Hadoop 2.2+ / CDH5.0+, but does not support CDH4, which is somewhere in between. -Sandy On Fri,

Re: Reading from .bz2 files with Spark

2014-05-16 Thread Xiangrui Meng
Hi Andrew, I submitted a patch and verified it solves the problem. You can download the patch from https://issues.apache.org/jira/browse/HADOOP-10614 . Best, Xiangrui On Fri, May 16, 2014 at 6:48 PM, Xiangrui Meng wrote: > Hi Andrew, > > This is the JIRA I created: > https://issues.apache.org/j

Re: Distribute jar dependencies via sc.AddJar(fileName)

2014-05-16 Thread DB Tsai
After reading the spark code more carefully, spark does `Thread.currentThread().setContextClassLoader` to the custom classloader. However, the classes have to be used via reflection with this approach. See, http://stackoverflow.com/questions/7452411/thread-currentthread-setcontextclassloader-with

Historical Data as Stream

2014-05-16 Thread Laeeq Ahmed
Hi, I have data in a file. Can I read it as Stream in spark? I know it seems odd to read file as stream but it has practical applications in real life if I can read it as stream. It there any other tools which can give this file as stream to Spark or I have to make batches manually which is not

Re: How to use spark-submit

2014-05-16 Thread Andrew Or
What kind of cluster mode are you running on? You may need to specify the jar through --jar, though we're working on making spark-submit automatically add the provided jar on the class path so we don't run into ClassNotFoundException as you have. What is the command that you ran? On Tue, May 6,

Re: How to pass config variables to workers

2014-05-16 Thread Theodore Wong
Sorry, yes, you are right, the documentation does indeed explain that setting spark.* options is the way to pass Spark configuration options to workers. Additionally, we've use the same mechanism to pass application-specific configuration options to workers; the "hack" part is naming our applicatio

Re: What is the difference between a Spark Worker and a Spark Slave?

2014-05-16 Thread Andrew Ash
They are different terminology for the same thing and should be interchangeable. On Fri, May 16, 2014 at 2:02 PM, Robert James wrote: > What is the difference between a Spark Worker and a Spark Slave? >

Re: Express VMs - good idea?

2014-05-16 Thread Mayur Rustagi
Frankly if you can give enough CPU performance to VM it should be good... but for development setting up locally is better 1. debuggable in IDE 2. Faster 3. samples like run-example etc Mayur Rustagi Ph: +1 (760) 203 3257 http://www.sigmoidanalytics.com @mayur_rustagi

Re: Standalone client failing with docker deployed cluster

2014-05-16 Thread Bharath Ravi Kumar
(Trying to bubble up the issue again...) Any insights (based on the enclosed logs) into why standalone client invocation might fail while issuing jobs through the spark console succeeded? Thanks, Bharath On Thu, May 15, 2014 at 5:08 PM, Bharath Ravi Kumar wrote: > Hi, > > I'm running the spark

Re: Reading from .bz2 files with Spark

2014-05-16 Thread Xiangrui Meng
Hi Andre, I could reproduce the bug with Hadoop 2.2.0. Some older version of Hadoop do not support splittable compression, so you ended up with sequential reads. It is easy to reproduce the bug with the following setup: 1) Workers are configured with multiple cores. 2) BZip2 files are big enough

Re: NoSuchMethodError: breeze.linalg.DenseMatrix

2014-05-16 Thread Xiangrui Meng
It doesn't work if you put the netlib-native jar inside an assembly jar. Try to mark it "provided" in the dependencies, and use --jars to include them with spark-submit. -Xiangrui On Wed, May 14, 2014 at 6:12 PM, wxhsdp wrote: > Hi, DB > > i tried including breeze library by using spark 1.0, it

Re: Passing runtime config to workers?

2014-05-16 Thread DB Tsai
Since the evn variables in driver will not be passed into workers, the most easy way you can do is refer to the variables directly in workers from driver. For example, val variableYouWantToUse = System.getenv("something defined in env") rdd.map( you can access `variableYouWantToUse` here ) Si

Re: Schema view of HadoopRDD

2014-05-16 Thread Mayur Rustagi
so you can use a input output format & read it whichever way you write... You can additionally provide variables in hadoop configuration to configure. Mayur Rustagi Ph: +1 (760) 203 3257 http://www.sigmoidanalytics.com @mayur_rustagi On Thu, May 8, 2014 at

Re: Reading from .bz2 files with Spark

2014-05-16 Thread Xiangrui Meng
Hi Andrew, This is the JIRA I created: https://issues.apache.org/jira/browse/MAPREDUCE-5893 . Hopefully someone wants to work on it. Best, Xiangrui On Fri, May 16, 2014 at 6:47 PM, Xiangrui Meng wrote: > Hi Andre, > > I could reproduce the bug with Hadoop 2.2.0. Some older version of > Hadoop d

Re: Benchmarking Spark with YCSB

2014-05-16 Thread Jay Vyas
I'm not sure what you mean... YCSB is for transactional systems. Spark isnt really in that category - its an analytics platform. RDDs by their very nature are not transactional., On Fri, May 16, 2014 at 6:37 AM, bhusted wrote: > Can anyone comment on what it would take to run Spark with YCSB

Re: A new resource for getting examples of Spark RDD API calls

2014-05-16 Thread zhen
Thanks for the suggestion. I will look into this. Zhen -- View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/A-new-resource-for-getting-examples-of-Spark-RDD-API-calls-tp5529p5532.html Sent from the Apache Spark User List mailing list archive at Nabble.com.

Re: SparkContext startup time out

2014-05-16 Thread Sophia
How did you deal with this problem finally?I also met with it. Best regards, -- View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/SparkContext-startup-time-out-tp1753p5739.html Sent from the Apache Spark User List mailing list archive at Nabble.com.

Re: different in spark on yarn mode and standalone mode

2014-05-16 Thread Sandy Ryza
We made several stabilization changes to Spark on YARN that made it into Spark 0.9.1 and CDH5.0. 1.0 significantly simplifies submitting a Spark app to a YARN cluster (wildly different invocations are no longer needed for yarn-client and yarn-cluster mode). I'm not sure about who is running it in

Re: Understanding epsilon in KMeans

2014-05-16 Thread Brian Gawalt
Hi Stuti, I think you're right. The epsilon parameter is indeed used as a threshold for deciding when KMeans has converged. If you look at line 201 of mllib's KMeans.scala: https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala#L201 you ca

Re: Understanding epsilon in KMeans

2014-05-16 Thread Krishna Sankar
Stuti, - The two numbers at different contexts, but finally end up in two sides of an && operator. - A parallel K-Means consists of multiple iterations which in turn consists of moving centroids around. A centroids would be deemed stabilized when the root square distance between suc

Re: Historical Data as Stream

2014-05-16 Thread Soumya Simanta
File is just a steam with a fixed length. Usually streams don't end but in this case it would. On the other hand if you real your file as a steam may not be able to use the entire data in the file for your analysis. Spark (give enough memory) can process large amounts of data quickly. > On M

Re: Is there any problem on the spark mailing list?

2014-05-16 Thread ssimanta
Same here. I've posted a bunch of questions in the last few days and they don't show up here and I'm also not getting email to my (gmail.com) account. I came here to post directly on the mailing list but saw this thread instead. At least, I'm not alone. -- View this message in context: http://

Re: Debugging Spark AWS S3

2014-05-16 Thread Ian Ferreira
Did you check the executor stderr logs? On 5/16/14, 2:37 PM, "Robert James" wrote: >I have Spark code which runs beautifully when MASTER=local. When I >run it with MASTER set to a spark ec2 cluster, the workers seem to >run, but the results, which are supposed to be put to AWS S3, don't >appear

Re: java serialization errors with spark.files.userClassPathFirst=true

2014-05-16 Thread Koert Kuipers
i do not think the current solution will work. i tried writing a version of ChildExecutorURLClassLoader that does have a proper parent and has a modified loadClass to reverse the order of parent and child in finding classes, and that seems to work, but now classes like SparkEnv are loaded by the ch

Re: How to pass config variables to workers

2014-05-16 Thread Andrew Or
Not a hack, this is documented here: http://spark.apache.org/docs/0.9.1/configuration.html, and is in fact the proper way of setting per-application Spark configurations. Additionally, you can specify default Spark configurations so you don't need to manually set it for all applications. If you ar

Re: Proper way to create standalone app with custom Spark version

2014-05-16 Thread Soumya Simanta
Install your custom spark jar to your local maven or ivy repo. Use this custom jar in your pom/sbt file. > On May 15, 2014, at 3:28 AM, Andrei wrote: > > (Sorry if you have already seen this message - it seems like there were some > issues delivering messages to the list yesterday) > > We

Re: Express VMs - good idea?

2014-05-16 Thread Matei Zaharia
Hey Marco, if you’re primarily interested in trying Spark, you can also just get a binary build from Apache: http://spark.apache.org/downloads.html. You only need Java on your machine to run it. To see it work with the rest of the Hadoop ecosystem components it is probably better to use a VM. M

Re: Using String Dataset for Logistic Regression

2014-05-16 Thread praveshjain1991
Thank you for your reply. So i take it that there's no direct way of using String datasets while using LR in Spark. -Pravesh -- View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/Using-String-Dataset-for-Logistic-Regression-tp5523p5810.html Sent from the Apache

Debugging Spark AWS S3

2014-05-16 Thread Robert James
I have Spark code which runs beautifully when MASTER=local. When I run it with MASTER set to a spark ec2 cluster, the workers seem to run, but the results, which are supposed to be put to AWS S3, don't appear on S3. I'm at a loss for how to debug this. I don't see any S3 exceptions anywhere. Ca

help me: Out of memory when spark streaming

2014-05-16 Thread Francis . Hu
hi, All I encountered OOM when streaming. I send data to spark streaming through Zeromq at a speed of 600 records per second, but the spark streaming only handle 10 records per 5 seconds( set it in streaming program) my two workers have 4 cores CPU and 1G RAM. These workers always occur Out

Re: Dead lock running multiple Spark jobs on Mesos

2014-05-16 Thread Martin Weindel
Andrew, thanks for your response. When using the coarse mode, the jobs run fine. My problem is the fine-grained mode. Here the parallel jobs nearly always end in a dead lock. It seems to have something to do with resource allocation, as Mesos shows neither used nor idle CPU resources in this

Re: Distribute jar dependencies via sc.AddJar(fileName)

2014-05-16 Thread DB Tsai
Hi Xiangrui, We're still using Spark 0.9 branch, and our job is submitted by ./bin/spark-class org.apache.spark.deploy.yarn.Client \ --jar \ --class \ --args \ --num-workers \ --master-class --master-memory \ --worker-memory \ --addJars Based on my understanding of the c

Re: How to pass config variables to workers

2014-05-16 Thread Theodore Wong
I found that the easiest way was to pass variables in the Spark configuration object. The only catch is that all of your properties keys must being with "spark." in order for Spark to propagate the values. So, for example, in the driver: SparkConf conf = new SparkConf(); conf.set("spark.myapp.mypr

Re: Turn BLAS on MacOSX

2014-05-16 Thread neville.lyh
I've had similar problems before and the following sbt option fixed it. sbt -J"-Dcom.github.fommil.netlib.BLAS=com.github.fommil.netlib.NativeRefBLAS" run Also you might need blas from homebrew. On Thu, May 15, 2014 at 10:50 AM, Debasish Das [via Apache Spark User List] wrote: > Hi, > > How d

Proper way to create standalone app with custom Spark version

2014-05-16 Thread Andrei
(Sorry if you have already seen this message - it seems like there were some issues delivering messages to the list yesterday) We can create standalone Spark application by simply adding "spark-core_2.x" to build.sbt/pom.xml and connecting it to Spark master. We can also build custom version of S

Re: Reading from .bz2 files with Spark

2014-05-16 Thread Xiangrui Meng
Hi Andrew, I verified that this is due to thread safety. I changed SPARK_WORKER_CORES to 1 in spark-env.sh, so there is only 1 thread per worker. Then I can load the file without any problem with different values of minPartitions. I will submit a JIRA to both Spark and Hadoop. Best, Xiangrui On

Re: different in spark on yarn mode and standalone mode

2014-05-16 Thread Sandy Ryza
Hi Vipul, Some advantages of using YARN: * YARN allows you to dynamically share and centrally configure the same pool of cluster resources between all frameworks that run on YARN. You can throw your entire cluster at a MapReduce job, then use some of it on an Impala query and the rest on Spark ap

Re: Reading from .bz2 files with Spark

2014-05-16 Thread Andrew Ash
Hi Xiangrui, // FYI I'm getting your emails late due to the Apache mailing list outage I'm using CDH4.4.0, which I think uses the MapReduce v2 API. The .jars are named like this: hadoop-hdfs-2.0.0-cdh4.4.0.jar I'm also glad you were able to reproduce! Please paste a link to the Hadoop bug you

Re: Is there any problem on the spark mailing list?

2014-05-16 Thread darkjh
Same thing here. There must be a problem ... I tried also send to user-subscr...@spark.apache.org or user-unsubstribe but no response. -- View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/Is-there-any-problem-on-the-spark-mailing-list-tp5509p5520.html Sent from t

Re: Spark unit testing best practices

2014-05-16 Thread Nan Zhu
+1, at least with current code just watch the log printed by DAGScheduler… -- Nan Zhu On Wednesday, May 14, 2014 at 1:58 PM, Mark Hamstra wrote: > serDe

Re: problem with hdfs access in spark job

2014-05-16 Thread Marcelo Vanzin
Hi Marcin, On Wed, May 14, 2014 at 7:22 AM, Marcin Cylke wrote: > - This looks like some problems with HA - but I've checked namenodes during > the job was running, and there > was no switch between master and slave namenode. > > 14/05/14 15:25:44 ERROR security.UserGroupInformation: > Priviled

Re: SparkContext startup time out

2014-05-16 Thread Sophia
How did you deal with this problem, I have met with it these days.God bless me. Best regard, -- View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/SparkContext-startup-time-out-tp1753p5738.html Sent from the Apache Spark User List mailing list archive at Nabble.c

Re: Reading from .bz2 files with Spark

2014-05-16 Thread Andre Bois-Crettez
We never saw your exception when reading bzip2 files with spark. But when we wrongly compiled spark against older version of hadoop (was default in spark), we ended up with sequential reading of bzip2 file, not taking advantage of block splits to work in parallel. Once we compiled spark with SPAR

Re: Using String Dataset for Logistic Regression

2014-05-16 Thread Brian Gawalt
Pravesh, Correct, the logistic regression engine is set up to perform classification tasks that take feature vectors (arrays of real-valued numbers) that are given a class label, and learning a linear combination of those features that divide the classes. As the above commenters have mentioned, th

Re: is Mesos falling out of favor?

2014-05-16 Thread Christopher Nguyen
Paco, that's a great video reference, thanks. To be fair to our friends at Yahoo, who have done a tremendous amount to help advance the cause of the BDAS stack, it's not FUD coming from them, certainly not in any organized or intentional manner. In vacuo we prefer Mesos ourselves, but also can't

Re: filling missing values in a sequence

2014-05-16 Thread Sean Owen
Not sure if this is feasible, but this literally does what I think you are describing: sc.parallelize(rdd1.first to rdd1.last) On Tue, May 13, 2014 at 4:56 PM, Mohit Jaggi wrote: > Hi, > I am trying to find a way to fill in missing values in an RDD. The RDD is a > sorted sequence. > For example,

Advanced log processing

2014-05-16 Thread Laurent T
Hi, I have some complex behavior i'd like to be advised on as i'm really new to Spark. I'm reading some log files that contains various events. There are two types of events: parents and children. A child event can only have one parent and a parent can have multiple children. Currently i'm mappi

Re: Schema view of HadoopRDD

2014-05-16 Thread Michael Armbrust
Here is a link with more info: http://people.apache.org/~pwendell/catalyst-docs/sql-programming-guide.html On Wed, May 7, 2014 at 10:09 PM, Debasish Das wrote: > Hi, > > For each line that we read as textLine from HDFS, we have a schema..if > there is an API that takes the schema as List[Symbol]

Re: Schema view of HadoopRDD

2014-05-16 Thread Mayur Rustagi
I guess what you are trying to do is get a columnar projection on your data, sparksql maybe a good option for you (especially if your data is sparse & good for columnar projection). If you are looking to work with simple key value then you are better off using Hbase input reader in hadoopIO & get

Re: different in spark on yarn mode and standalone mode

2014-05-16 Thread Vipul Pandey
Thanks for responding, Sandy. YARN for sure is a more mature way of working on shared resources. I was not sure about how stable Spark on YARN is and if anyone is using it in production. I have been using Standalone mode in our dev cluster but multi-tenancy and resource allocation wise it's di

RE: slf4j and log4j loop

2014-05-16 Thread Adrian Mocanu
Please ignore. This was sent last week not sure why it arrived so late. -Original Message- From: amoc [mailto:amoc...@verticalscope.com] Sent: May-09-14 10:13 AM To: u...@spark.incubator.apache.org Subject: Re: slf4j and log4j loop Hi Patrick/Sean, Sorry to resurrect this thread, but aft

What does Spark cache() actually do?

2014-05-16 Thread PengWeiPRC
Hi there, I was wondering if some one could explain me how the cache() function works in Spark in these phases: (1) If I have a huge file, say 1TB, which cannot be entirely stored in Memory. What will happen if I try to create a RDD of this huge file and "cache"? (2) If it works in Spark, it ca

Re: Spark unit testing best practices

2014-05-16 Thread Andras Nemeth
Thanks for the answers! On a concrete example, here is what I did to test my (wrong :) ) hypothesis before writing my email: class SomethingNotSerializable { def process(a: Int): Int = 2 *a } object NonSerializableClosure extends App { val sc = new spark.SparkContext( "local", "Ser

Re: Distribute jar dependencies via sc.AddJar(fileName)

2014-05-16 Thread DB Tsai
The jars are actually there (and in classpath), but you need to load through reflection. I've another thread giving the workaround. Sincerely, DB Tsai --- My Blog: https://www.dbtsai.com LinkedIn: https://www.linkedin.com/in/dbtsai On Fri, Ma

Re: Real world

2014-05-16 Thread Bertrand Dechoux
http://spark-summit.org ? Bertrand On Thu, May 8, 2014 at 2:05 AM, Ian Ferreira wrote: > Folks, > > I keep getting questioned on real world experience of Spark as in mission > critical production deployments. Does anyone have some war stories to share > or know of resources to review? > > Cheer

Re: different in spark on yarn mode and standalone mode

2014-05-16 Thread Vipul Pandey
And I thought I sent it to the right list! Here you go again - Question below : On May 14, 2014, at 3:06 PM, Vipul Pandey wrote: > So here's a followup question : What's the preferred mode? > We have a new cluster coming up with petabytes of data and we intend to take > Spark to production. W

Re: java serialization errors with spark.files.userClassPathFirst=true

2014-05-16 Thread Koert Kuipers
well, i modified ChildExecutorURLClassLoader to also delegate to parentClassloader if NoClassDefFoundError is thrown... now i get yet another error. i am clearly missing something with these classloaders. such nasty stuff... giving up for now. just going to have to not use spark.files.userClassPath

unsubscribe

2014-05-16 Thread eric perler

Re: Hadoop 2.3 Centralized Cache vs RDD

2014-05-16 Thread hequn cheng
I tried centralized cache step by step following the apache hadoop oficial website, but it seems centralized cache doesn't work. see : http://stackoverflow.com/questions/22293358/centralized-cache-failed-in-hadoop-2-3 . Can anyone succeed? 2014-05-15 5:30 GMT+08:00 William Kang : > Hi, > Any com

java serialization errors with spark.files.userClassPathFirst=true

2014-05-16 Thread Koert Kuipers
when i set spark.files.userClassPathFirst=true, i get java serialization errors in my tasks, see below. when i set userClassPathFirst back to its default of false, the serialization errors are gone. my spark.serializer is KryoSerializer. the class org.apache.hadoop.fs.Path is in the spark assembly

Re: cant get tests to pass anymore on master master

2014-05-16 Thread Koert Kuipers
yeah sure. it is ubuntu 12.04 with jdk1.7.0_40 what else is relevant that i can provide? On Thu, May 15, 2014 at 12:17 PM, Sean Owen wrote: > FWIW I see no failures. Maybe you can say more about your environment, etc. > > On Wed, May 7, 2014 at 10:01 PM, Koert Kuipers wrote: > > i used to be a

Re: Understanding epsilon in KMeans

2014-05-16 Thread Long Pham
Stuti, I'm answering your questions in order: 1. From MLLib https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala#L159 *,* you can see that clustering stops when we have reached*maxIterations* or there are no more*activeRuns*. KMeans is e

Nested method in a class: Task not serializable?

2014-05-16 Thread Pierre B
Hi! I understand the usual "Task not serializable" issue that arises when accessing a field or a method that is out of scope of a closure. To fix it, I usually define a local copy of these fields/methods, which avoids the need to serialize the whole class: class MyClass(val myField: Any) { def

What is the difference between a Spark Worker and a Spark Slave?

2014-05-16 Thread Robert James
What is the difference between a Spark Worker and a Spark Slave?

Re: Counting things only once

2014-05-16 Thread Mark Hamstra
https://spark-project.atlassian.net/browse/SPARK-732 On Fri, May 16, 2014 at 9:05 AM, Daniel Siegmann wrote: > I want to use accumulators to keep counts of things like invalid lines > found and such, for reporting purposes. Similar to Hadoop counters. This > may seem simple, but my case is a bit

Re: Distribute jar dependencies via sc.AddJar(fileName)

2014-05-16 Thread Robert James
I've experienced the same bug, which I had to workaround manually. I posted the details here: http://stackoverflow.com/questions/23687081/spark-workers-unable-to-find-jar-on-ec2-cluster On 5/15/14, DB Tsai wrote: > Hi guys, > > I think it maybe a bug in Spark. I wrote some code to demonstrate th

Passing runtime config to workers?

2014-05-16 Thread Robert James
What is a good way to pass config variables to workers? I've tried setting them in environment variables via spark-env.sh, but, as far as I can tell, the environment variables set there don't appear in workers' environments. If I want to be able to configure all workers, what's a good way to do i

Re: Counting things only once

2014-05-16 Thread Mark Hamstra
Better, the current location: https://issues.apache.org/jira/browse/SPARK-732 On Fri, May 16, 2014 at 1:47 PM, Mark Hamstra wrote: > https://spark-project.atlassian.net/browse/SPARK-732 > > > On Fri, May 16, 2014 at 9:05 AM, Daniel Siegmann > wrote: > >> I want to use accumulators to keep count

Re: How to run the SVM and LogisticRegression

2014-05-16 Thread Debasish Das
There are examples to run them in BinaryClassification.scala in org.apache.spark.examples... On Wed, May 14, 2014 at 1:36 PM, yxzhao wrote: > > Hello, > I found the classfication algorithms SVM and LogisticRegression implemented > in the following directory. And how to run them? What is the co

writing my own RDD

2014-05-16 Thread Koert Kuipers
in writing my own RDD i ran into a few issues with respect to stuff being private in spark. in compute i would like to return an iterator that respects task killing (as HadoopRDD does), but the mechanics for that are inside the private InterruptibleIterator. also the exception i am supposed to thr

Re: Distribute jar dependencies via sc.AddJar(fileName)

2014-05-16 Thread DB Tsai
Hi guys, I think it maybe a bug in Spark. I wrote some code to demonstrate the bug. Example 1) This is how Spark adds jars. Basically, add jars to cutomURLClassLoader. https://github.com/dbtsai/classloader-experiement/blob/master/calling/src/main/java/Calling1.java It doesn't work for two reaso

Re: is Mesos falling out of favor?

2014-05-16 Thread Gerard Maas
Regarding docs, Andrew Ash recently did a great effort in refreshing the Spark on Mesos documentation. https://github.com/apache/spark/pull/756 It will become part of 1.0 -kr, Gerard. On Fri, May 9, 2014 at 3:46 PM, Tim St Clair wrote: > > > > > - Original Message - > > From: "deri

Re: KryoSerializer Exception

2014-05-16 Thread Andrea Esposito
UP, doesn't anyone know something about it? ^^ 2014-05-06 12:05 GMT+02:00 Andrea Esposito : > Hi there, > > sorry if i'm posting a lot lately. > > i'm trying to add the KryoSerializer but i receive this exception: > 2014 - 05 - 06 11: 45: 23 WARN TaskSetManager: 62 - Loss was due to > java.io.EO

Doubts regarding Shark

2014-05-16 Thread vinay Bajaj
Hello I have few questions regarding shark. 1) I have a table of 60 GB and i have total memory of 50 GB but when i try to cache the table it get cached successfully. How shark caches the table there was not enough memory to get the table in memory. And how cache eviction policies (FIFO and LRU) w

RE: same log4j slf4j error in spark 9.1

2014-05-16 Thread Adrian Mocanu
Hi guys, This has been solved. These emails are from last week when the mailing list didn’t work. From: Tathagata Das [mailto:tathagata.das1...@gmail.com] Sent: May-15-14 4:50 PM To: user@spark.apache.org Cc: u...@spark.incubator.apache.org Subject: Re: same log4j slf4j error in spark 9.1 Spark

Variables outside of mapPartitions scope

2014-05-16 Thread pedro
I am working on some code which uses mapPartitions. Its working great, except when I attempt to use a variable within the function passed to mapPartitions which references something outside of the scope (for example, a variable declared immediately before the mapPartitions call). When this happens,

Re: How to read a multipart s3 file?

2014-05-16 Thread Nicholas Chammas
On Wed, May 7, 2014 at 4:44 PM, Aaron Davidson wrote: Spark can only run as many tasks as there are partitions, so if you don't > have enough partitions, your cluster will be underutilized. This is a very important point. kamatsuoka, how many partitions does your RDD have when you try to save

Re: Understanding epsilon in KMeans

2014-05-16 Thread Xiangrui Meng
In Spark's KMeans, if no cluster center moves more than epsilon in Euclidean distance from previous iteration, the algorithm finishes. No further iterations are performed. For Mahout, you need to check the documentation or the code to see what epsilon means there. -Xiangrui On Wed, May 14, 2014 at

Problem when sorting big file

2014-05-16 Thread Gustavo Enrique Salazar Torres
Hi there: I have this dataset (about 12G) which I need to sort by key. I used the sortByKey method but when I try to save the file to disk (HDFS in this case) it seems that some tasks run out of time because they have too much data to save and it can't fit in memory. I say this because before the

Re: accessing partition i+1 from mapper of partition i

2014-05-16 Thread Brian Gawalt
I don't think there's a direct way of bleeding elements across partitions. But you could write it yourself relatively succinctly: A) Sort the RDD B) Look at the sorted RDD's partitions with the .mapParititionsWithIndex( ) method. Map each partition to its partition ID, and its maximum element. Col

Re: java serialization errors with spark.files.userClassPathFirst=true

2014-05-16 Thread Koert Kuipers
ok i think the issue is visibility: a classloader can see all classes loaded by its parent classloader. but userClassLoader does not have a parent classloader, so its not able to "see" any classes that parentLoader is responsible for. in my case userClassLoader is trying to get AvroInputFormat whic

Re: Hadoop 2.3 Centralized Cache vs RDD

2014-05-16 Thread Bertrand Dechoux
http://hadoop.apache.org/docs/r2.3.0/hadoop-project-dist/hadoop-hdfs/CentralizedCacheManagement.html We do not currently cache blocks which are under construction, corrupt, or otherwise incomplete. Have you tried with a file with more than 1 block? And dfs.namenode.path.based.cache.refresh.inter

Counting things only once

2014-05-16 Thread Daniel Siegmann
I want to use accumulators to keep counts of things like invalid lines found and such, for reporting purposes. Similar to Hadoop counters. This may seem simple, but my case is a bit more complicated. The code which is creating an RDD from a transform is separated from the code which performs the op

spark-submit / S3

2014-05-16 Thread Nick Pentreath
Hi I see from the docs for 1.0.0 that the new "spark-submit" mechanism seems to support specifying the jar with hdfs:// or http:// Does this support S3? (It doesn't seem to as I have tried it on EC2 but doesn't seem to work): ./bin/spark-submit --master local[2] --class myclass s3n://bucket/myap

Re: java serialization errors with spark.files.userClassPathFirst=true

2014-05-16 Thread Koert Kuipers
ok i put lots of logging statements in the ChildExecutorURLClassLoader. this is what i see: * the urls for userClassLoader are correct and includes only my one jar. * for one class that only exists in my jar i see it gets loaded correctly using userClassLoader * for a class that exists in both m

Re: java serialization errors with spark.files.userClassPathFirst=true

2014-05-16 Thread Koert Kuipers
after removing all class paramater of class Path from my code, i tried again. different but related eror when i set spark.files.userClassPathFirst=true now i dont even use FileInputFormat directly. HadoopRDD does... 14/05/16 12:17:17 ERROR Executor: Exception in task ID 45 java.lang.NoClassDefFou

Error while launching ec2 spark cluster with HVM (r3.large)

2014-05-16 Thread Usman Ghani
Hi All, I am trying to use amazon memory optimized (R3) instances in the Oregon region. I am getting 'Connection refused' during the SSH setup phase. I tried using both root and ec2-user as user ids. sh: connect to host ec2-54-185-57-74.us-west-2.compute.amazonaws.com port 22: Connection refused

Re: Express VMs - good idea?

2014-05-16 Thread Sean Owen
Hey Marco, I tried the CDH5 VM today and it works fine -- but note that you need to start the Spark service after the VM boots. Just go to CM and choose Start from the dropdown next to Spark. spark-shell works fine then. On Wed, May 14, 2014 at 1:00 PM, Marco Shaw wrote: > Hi, > > I've wanted to

Re: Reading from .bz2 files with Spark

2014-05-16 Thread Xiangrui Meng
Hi Andrew, Could you try varying the minPartitions parameter? For example: val r = sc.textFile("/user/aa/myfile.bz2", 4).count val r = sc.textFile("/user/aa/myfile.bz2", 8).count Best, Xiangrui On Tue, May 13, 2014 at 9:08 AM, Xiangrui Meng wrote: > Which hadoop version did you use? I'm not su

Re: Understanding epsilon in KMeans

2014-05-16 Thread Sean Owen
It is running k-means many times, independently, from different random starting points in order to pick the best clustering. Convergence ends one run, not all of them. Yes epsilon should be the same as "convergence threshold" elsewhere. You can set epsilon if you instantiate KMeans directly. Mayb

Re: cant get tests to pass anymore on master master

2014-05-16 Thread Koert Kuipers
i tried on a few different machines, including a server, all same ubuntu and same java, and got same errors. i also tried modifying the timeouts in the unit tests and it did not help. ok i will try blowing away local maven repo and do clean. On Thu, May 15, 2014 at 12:49 PM, Sean Owen wrote: >

Re: JavaNetworkWordCount

2014-05-16 Thread Mayur Rustagi
It would look ugly.. as explicit datatypes need to be mentioned.. you are better off using parallelize instead. Mayur Rustagi Ph: +1 (760) 203 3257 http://www.sigmoidanalytics.com @mayur_rustagi On Fri, May 16, 2014 at 6:11 PM, Eduardo Costa Alfaia < e.costaa

How to pass config variables to workers

2014-05-16 Thread srobertjames
What is a good way to pass config variables to workers? I've tried setting them in environment variables via spark-env.sh, but, as far as I can tell, the environment variables set there don't appear in workers' environments. If I want to be able to configure all workers, what's a good way to do i

Spark with Drill

2014-05-16 Thread N . Venkata Naga Ravi
Hi, I am trying to understand and and seeing Drill as one of the upcoming interesting tool outside. Can somebody clarify where Drill is going to position in Hadoop ecosystem compare with Spark and Shark? Is it going to be used as alternative to any one of the Spark/Shark or Storm? Or Drill can

Re: Efficient implementation of getting top 10 hashtags in last 5 mins window

2014-05-16 Thread bgawalt
Hi nilmish, One option for you is to consider moving to a different algorithm. The SpaceSaver/StreamSummary method will get you approximate results in exchange for smaller data structure size. It has an implementation in Twitter's Algebird library, if you're using Scala: https://github.com/twitte

Re: filling missing values in a sequence

2014-05-16 Thread bgawalt
Hello Mohit, I don't think there's a direct way of bleeding elements across partitions. But you could write it yourself relatively succinctly: A) Sort the RDD B) Look at the sorted RDD's partitions with the .mapParititionsWithIndex( ) method. Map each partition to its partition ID, and its maximu

JavaNetworkWordCount

2014-05-16 Thread Eduardo Costa Alfaia
Hi Guys, TD has given me this piece of code: “sparkContext.makeRDD(1 to 100, 100).collect()", I am using a java code of NetworkWordcount, How could I use this piece in this code in java? Thanks -- Informativa sulla Privacy: http://www.unibs.it/node/8155

Re: Job failed: java.io.NotSerializableException: org.apache.spark.SparkContext

2014-05-16 Thread Nathan Kronenfeld
Serializing the main object isn't going to help here - it's SparkContext it's complaining about. The problem is that the context is, according to the code you sent, computeDwt has a signature of: class DWTSample ... { def computDWT (sc: SparkContext, data: ArrayBuffer[(Int, Double)]): List[Do

Re: Express VMs - good idea?

2014-05-16 Thread Stephen Boesch
Hi Marco, Hive itself is not working in the CDH5.0 VM (due to FNFE's on the third party jars). While you did not mention using Shark, you may keep that in mind. I will try out spark-only commands late today and report what I find. 2014-05-14 5:00 GMT-07:00 Marco Shaw : > Hi, > > I've wanted t

Workers unable to find class, even when in the SparkConf JAR list

2014-05-16 Thread Robert James
I'm using spark-ec2 to run some Spark code. When I set master to "local", then it runs fine. However, when I set master to $MASTER, the workers immediately fail, with java.lang.NoClassDefFoundError for the classes. I've used sbt-assembly to make a jar with the classes, confirmed using jar tvf th

  1   2   >