Hadoop 2.3 Centralized Cache vs RDD

2014-05-16 Thread William Kang
Hi, Any comments or thoughts on the implications of the newly released feature from Hadoop 2.3 on the centralized cache? How different it is from RDD? Many thanks. Cao

Re: maven for building scala simple program

2014-05-16 Thread Laeeq Ahmed
Hi Ryan, It worked like a charm. Much appreciated. Laeeq. On Wednesday, May 7, 2014 1:30 AM, Ryan Compton wrote: I've been using this (you'll need maven 3). http://maven.apache.org/POM/4.0.0"; xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";         xsi:schemaLocation="http://maven.ap

Benchmarking Spark with YCSB

2014-05-16 Thread bhusted
Can anyone comment on what it would take to run Spark with YCSB and HBase for benchmarking? Has this been done before or been reviewed? If not, our company is looking to make an investment to perform the development necessary but we are looking for any technical insight on what it would take to m

Re: Spark GCE Script

2014-05-16 Thread Akhil Das
Hi I have sent a pull request https://github.com/apache/spark/pull/681 you can verify it and add it :) Thanks Best Regards On Thu, May 8, 2014 at 2:58 AM, Aureliano Buendia wrote: > Please send a pull request, this should be maintained by the community, > just in case you do not feel like con

cant get tests to pass anymore on master master

2014-05-16 Thread Koert Kuipers
i used to be able to get all tests to pass. with java 6 and sbt i get PermGen errors (no matter how high i make the PermGen). so i have given up on that. with java 7 i see 1 error in a bagel test and a few in streaming tests. any ideas? see the error in BagelSuite below. [info] - large number of

Re: Spark workers keep getting disconnected(Keep dying) from the cluster.

2014-05-16 Thread akeed
Got the same experience over here. 0.9.1 (not from github, from official download page), running hadoop 2.2. -- View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/Spark-workers-keep-getting-disconnected-Keep-dying-from-the-cluster-tp5740p5747.html Sent from the Ap

Re: Error starting EC2 cluster

2014-05-16 Thread Aliaksei Litouka
Well... the reason was an out-of-date version of Python (2.6.6) on the machine where I ran the script. If anyone else experiences this issue - just update your Python. On Sun, May 4, 2014 at 7:51 PM, Aliaksei Litouka wrote: > I am using Spark 0.9.1. When I'm trying to start a EC2 cluster with t

Re: is Mesos falling out of favor?

2014-05-16 Thread Gerard Maas
By looking at your config, I think there's something wrong with your setup. One of the key elements of Mesos is that you are abstracted from where the execution of your task takes place. The SPARK_EXECUTOR_URI tells Mesos where to find the 'framework' (in Mesos jargon) required to execute a job. (

unsubscribe

2014-05-16 Thread eric perler

Re: Packaging a spark job using maven

2014-05-16 Thread Eugen Cepoi
Laurent the problem is that the reference.conf that is embedded in akka jars is being overriden by some other conf. This happens when multiple files have the same name. I am using Spark with maven. In order to build the fat jar I use the shade plugin and it works pretty well. The trick here is to u

Spark workers keep getting disconnected(Keep dying) from the cluster.

2014-05-16 Thread Ravi Hemnani
Hey, I am facing a weird issue. My spark workers keep dying every now and then and in the master logs i keep on seeing following messages, 14/05/14 10:09:24 WARN Master: Removing worker-20140514080546-x.x.x.x-50737 because we got no heartbeat in 60 seconds 14/05/14 14:18:41 WARN Master: Removi

Re: Spark LIBLINEAR

2014-05-16 Thread DB Tsai
Hi Deb, My co-worker fixed a owlqn bug in breeze, and it's important to have this to converge to the correct result. https://github.com/scalanlp/breeze/pull/247 You may want to use the snapshot of breeze to have this fix in. Sincerely, DB Tsai -

Re: Stable Hadoop version supported ?

2014-05-16 Thread Sandy Ryza
Hi Soumya, If you want to stick with CDH, CDH5.0 is the latest stable release. If you want to use an Apache release, I'd go with Hadoop 2.3. -Sandy On Wed, May 14, 2014 at 12:17 PM, Soumya Simanta wrote: > Currently I've HDFS with version hadoop0.20.2-cdh3u6 on Spark 0.9.1. I > want to upgrad

Re: Local Dev Env with Mesos + Spark Streaming on Docker: Can't submit jobs.

2014-05-16 Thread joe schmoo
I On May 5, 2014 1:12 PM, "Gerard Maas" wrote: > > Hi all, > > I'm currently working on creating a set of docker images to facilitate local development with Spark/streaming on Mesos (+zk, hdfs, kafka) > > After solving the initial hurdles to get things working together in docker containers, now ev

spark 0.9.1 textFile hdfs unknown host exception

2014-05-16 Thread Eugen Cepoi
Hi, I have some strange behaviour when using textFile to read some data from HDFS in spark 0.9.1. I get UnknownHost exceptions, where hadoop client tries to resolve the dfs.nameservices and fails. So far: - this has been tested inside the shell - the exact same code works with spark-0.8.1 - t

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

2014-05-16 Thread nilmish
I wanted to know how can we efficiently get top 10 hashtags in last 5 mins window. Currently I am using reduceByKeyAndWindow over 5 mins window and then sorting to get top 10 hashtags. But it is taking a lot of time. How can we do it efficiently ? -- View this message in context: http://apache-

Re: same log4j slf4j error in spark 9.1

2014-05-16 Thread Tathagata Das
Spark 0.9.1 does not depend on log4j-over-slf4j (hereis the SBT file for 0.9.1). Are you sure that no other dependency in you

Re: Spark LIBLINEAR

2014-05-16 Thread Tom Vacek
I've done some comparisons with my own implementation of TRON on Spark. From a distributed computing perspective, it does 2x more local work per iteration than LBFGS, so the parallel isoefficiency is improved slightly. I think the truncated Newton solver holds some potential because there have be

Re: Using String Dataset for Logistic Regression

2014-05-16 Thread DB Tsai
You could also use dummy coding to convert categorical feature to numeric feature. http://en.wikipedia.org/wiki/Categorical_variable#Dummy_coding Sincerely, DB Tsai --- My Blog: https://www.dbtsai.com LinkedIn: https://www.linkedin.com/in/dbtsa

Re: spark on yarn-standalone, throws StackOverflowError and fails somtimes and succeed for the rest

2014-05-16 Thread Xiangrui Meng
Could you try `println(result.toDebugString())` right after `val result = ...` and attach the result? -Xiangrui On Fri, May 9, 2014 at 8:20 AM, phoenix bai wrote: > after a couple of tests, I find that, if I use: > > val result = model.predict(prdctpairs) > result.map(x => > x.user+","+x.prod

count()-ing gz files gives java.io.IOException: incorrect header check

2014-05-16 Thread Nick Chammas
I’m trying to do a simple count() on a large number of GZipped files in S3. My job is failing with the following message: 14/05/15 19:12:37 WARN scheduler.TaskSetManager: Loss was due to java.io.IOException java.io.IOException: incorrect header check at org.apache.hadoop.io.compress.zlib.Zlib

Re: Equivalent of collect() on DStream

2014-05-16 Thread Sean Owen
Are you not just looking for the foreachRDD() method on DStream? http://spark.apache.org/docs/0.9.1/streaming-programming-guide.html#output-operations It gives you an RDD that you can do what you want with, including collect() it. On Thu, May 15, 2014 at 5:33 AM, Stephen Boesch wrote: > Looking

Re: Stable Hadoop version supported ?

2014-05-16 Thread Sean Owen
Although you need to compile it differently for different versions of HDFS / Hadoop, as far as I know Spark continues to work with Hadoop 1.x (and probably older 0.20.x as a result -- your experience is an existence proof.) And it works with the newest Hadoop 2.4.x, again with the appropriate build

Calling external classes added by sc.addJar needs to be through reflection

2014-05-16 Thread DB Tsai
Finally find a way out of the ClassLoader maze! It took me some times to understand how it works; I think it worths to document it in a separated thread. We're trying to add external utility.jar which contains CSVRecordParser, and we added the jar to executors through sc.addJar APIs. If the insta

Re: Schema view of HadoopRDD

2014-05-16 Thread Flavio Pompermaier
Is there any Spark plugin/add-on that facilitate the query to a JSON content? Best, Flavio On Thu, May 15, 2014 at 6:53 PM, Michael Armbrust wrote: > 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

Re: Equivalent of collect() on DStream

2014-05-16 Thread Tathagata Das
Doesnt DStream.foreach() suffice? anyDStream.foreach { rdd => // do something with rdd } On Wed, May 14, 2014 at 9:33 PM, Stephen Boesch wrote: > Looking further it appears the functionality I am seeking is in the > following *private[spark] * class ForEachdStream > > (version 0.8.1 , yes

unsubscribe

2014-05-16 Thread eric perler
unsubscribe

Re: How to run the SVM and LogisticRegression

2014-05-16 Thread Xiangrui Meng
If you check out the master branch, there are some examples that can be used as templates under examples/src/main/scala/org/apache/spark/examples/mllib Best, Xiangrui On Wed, May 14, 2014 at 1:36 PM, yxzhao wrote: > > Hello, > I found the classfication algorithms SVM and LogisticRegression impl

Re: Turn BLAS on MacOSX

2014-05-16 Thread Xiangrui Meng
Are you running Spark or just Breeze? First try breeze-natives locally with the reference blas library and see whether it works or not. Also, do not enable multi-threading when you compile OpenBLAS (USE_THREADS=0). -Xiangrui On Tue, May 13, 2014 at 2:17 AM, wxhsdp wrote: > Hi, Xiangrui > > i co

Standalone client failing with docker deployed cluster

2014-05-16 Thread Bharath Ravi Kumar
Hi, I'm running the spark server with a single worker on a laptop using the docker images. The spark shell examples run fine with this setup. However, a standalone java client that tries to run wordcount on a local files (1 MB in size), the execution fails with the following error on the stdout of

unsubscribe

2014-05-16 Thread eric perler
unsubscribe

advice on maintaining a production spark cluster?

2014-05-16 Thread Josh Marcus
Hey folks, I'm wondering what strategies other folks are using for maintaining and monitoring the stability of stand-alone spark clusters. Our master very regularly loses workers, and they (as expected) never rejoin the cluster. This is the same behavior I've seen using akka cluster (if that's w

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

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

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

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: 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

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

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

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

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: 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: 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: 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

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: 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

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

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

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

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

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: 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

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: 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

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

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: 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

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: 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

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: 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

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: 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

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

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: 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

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

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?

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

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

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

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

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

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

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: 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: 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: 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

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: 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

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: 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: 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: 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: 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: 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: 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: 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: 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: 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: 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: 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: 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 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

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: 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

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

  1   2   >