Re: Unable to redirect Spark logs to slf4j

2014-03-05 Thread Patrick Wendell
with compile scope helps, and I confirm the logging is redirected to slf4j/Logback correctly now with the patched module. I'm not sure however if using compile scope for slf4j-log4j12 is a good idea. -- Best regards, Sergey Parhomenko On 5 March 2014 20:11, Patrick Wendell pwend...@gmail.com

Re: Python 2.7 + numpy break sortByKey()

2014-03-06 Thread Patrick Wendell
The difference between your two jobs is that take() is optimized and only runs on the machine where you are using the shell, whereas sortByKey requires using many machines. It seems like maybe python didn't get upgraded correctly on one of the slaves. I would look in the /root/spark/work/ folder

Re: no stdout output from worker

2014-03-09 Thread Patrick Wendell
Hey Sen, Is your code in the driver code or inside one of the tasks? If it's in the tasks, the place you would expect these to be is in stdout file under spark/appid/work/[stdout/stderr]. Are you seeing at least stderr logs in that folder? If not then the tasks might not be running on the

Re: [External] Re: no stdout output from worker

2014-03-10 Thread Patrick Wendell
help. Ranjan On 3/9/14, 10:50 PM, Patrick Wendell pwend...@gmail.com wrote: Hey Sen, Is your code in the driver code or inside one of the tasks? If it's in the tasks, the place you would expect these to be is in stdout file under spark/appid/work/[stdout/stderr]. Are you seeing at least

Re: Too many open files exception on reduceByKey

2014-03-10 Thread Patrick Wendell
Hey Matt, The best way is definitely just to increase the ulimit if possible, this is sort of an assumption we make in Spark that clusters will be able to move it around. You might be able to hack around this by decreasing the number of reducers but this could have some performance implications

Re: Round Robin Partitioner

2014-03-13 Thread Patrick Wendell
In Spark 1.0 we've added better randomization to the scheduling of tasks so they are distributed more evenly by default. https://github.com/apache/spark/commit/556c56689bbc32c6cec0d07b57bd3ec73ceb243e However having specific policies like that isn't really supported unless you subclass the RDD

Re: slf4j and log4j loop

2014-03-16 Thread Patrick Wendell
This is not released yet but we're planning to cut a 0.9.1 release very soon (e.g. most likely this week). In the mean time you'll have checkout branch-0.9 of Spark and publish it locally then depend on the snapshot version. Or just wait it out... On Fri, Mar 14, 2014 at 2:01 PM, Adrian Mocanu

Re: combining operations elegantly

2014-03-23 Thread Patrick Wendell
at 9:39 PM, Koert Kuipers ko...@tresata.com wrote: not that long ago there was a nice example on here about how to combine multiple operations on a single RDD. so basically if you want to do a count() and something else, how to roll them into a single job. i think patrick wendell gave

Re: How many partitions is my RDD split into?

2014-03-23 Thread Patrick Wendell
As Mark said you can actually access this easily. The main issue I've seen from a performance perspective is people having a bunch of really small partitions. This will still work but the performance will improve if you coalesce the partitions using rdd.coalesce(). This can happen for example if

Re: No space left on device exception

2014-03-23 Thread Patrick Wendell
Ognen - just so I understand. The issue is that there weren't enough inodes and this was causing a No space left on device error? Is that correct? If so, that's good to know because it's definitely counter intuitive. On Sun, Mar 23, 2014 at 8:36 PM, Ognen Duzlevski og...@nengoiksvelzud.com wrote:

Re: How many partitions is my RDD split into?

2014-03-24 Thread Patrick Wendell
Ah we should just add this directly in pyspark - it's as simple as the code Shivaram just wrote. - Patrick On Mon, Mar 24, 2014 at 1:25 PM, Shivaram Venkataraman shivaram.venkatara...@gmail.com wrote: There is no direct way to get this in pyspark, but you can get it from the underlying java

Re: Error reading HDFS file using spark 0.9.0 / hadoop 2.2.0 - incompatible protobuf 2.5 and 2.4.1

2014-03-25 Thread Patrick Wendell
Starting with Spark 0.9 the protobuf dependency we use is shaded and cannot interfere with other protobuf libaries including those in Hadoop. Not sure what's going on in this case. Would someone who is having this problem post exactly how they are building spark? - Patrick On Fri, Mar 21, 2014

Re: Building Spark 0.9.x for CDH5 with mrv1 installation (Protobuf 2.5 upgrade)

2014-03-26 Thread Patrick Wendell
I'm not sure exactly how your cluster is configured. But as far as I can tell Cloudera's MR1 CDH5 dependencies are against Hadoop 2.3. I'd just find the exact CDH version you have and link against the `mr1` version of their published dependencies in that version. So I think you wan't

Re: Announcing Spark SQL

2014-03-27 Thread Patrick Wendell
Hey Rohit, I think external tables based on Cassandra or other datastores will work out-of-the box if you build Catalyst with Hive support. Michael may have feelings about this but I'd guess the longer term design for having schema support for Cassandra/HBase etc likely wouldn't rely on hive

Re: Spark webUI - application details page

2014-03-30 Thread Patrick Wendell
This will be a feature in Spark 1.0 but is not yet released. In 1.0 Spark applications can persist their state so that the UI can be reloaded after they have completed. - Patrick On Sun, Mar 30, 2014 at 10:30 AM, David Thomas dt5434...@gmail.com wrote: Is there a way to see 'Application

Re: Using ProtoBuf 2.5 for messages with Spark Streaming

2014-03-31 Thread Patrick Wendell
Spark now shades its own protobuf dependency so protobuf 2.4.1 should't be getting pulled in unless you are directly using akka yourself. Are you? Does your project have other dependencies that might be indirectly pulling in protobuf 2.4.1? It would be helpful if you could list all of your

Re: Using ProtoBuf 2.5 for messages with Spark Streaming

2014-04-01 Thread Patrick Wendell
Do you get the same problem if you build with maven? On Tue, Apr 1, 2014 at 12:23 PM, Vipul Pandey vipan...@gmail.com wrote: SPARK_HADOOP_VERSION=2.0.0-cdh4.2.1 sbt/sbt assembly That's all I do. On Apr 1, 2014, at 11:41 AM, Patrick Wendell pwend...@gmail.com wrote: Vidal - could you show

Re: Using ProtoBuf 2.5 for messages with Spark Streaming

2014-04-02 Thread Patrick Wendell
(default-cli) on project spark-0.9.0-incubating: Error reading assemblies: No assembly descriptors found. - [Help 1] upon runnning mvn -Dhadoop.version=2.0.0-cdh4.2.1 -DskipTests clean assembly:assembly On Apr 1, 2014, at 4:13 PM, Patrick Wendell pwend...@gmail.com wrote: Do you get the same

Re: Spark output compression on HDFS

2014-04-02 Thread Patrick Wendell
For textFile I believe we overload it and let you set a codec directly: https://github.com/apache/spark/blob/master/core/src/test/scala/org/apache/spark/FileSuite.scala#L59 For saveAsSequenceFile yep, I think Mark is right, you need an option. On Wed, Apr 2, 2014 at 12:36 PM, Mark Hamstra

Re: Resilient nature of RDD

2014-04-02 Thread Patrick Wendell
The driver stores the meta-data associated with the partition, but the re-computation will occur on an executor. So if several partitions are lost, e.g. due to a few machines failing, the re-computation can be striped across the cluster making it fast. On Wed, Apr 2, 2014 at 11:27 AM, David

Re: Is there a way to get the current progress of the job?

2014-04-02 Thread Patrick Wendell
Hey Phillip, Right now there is no mechanism for this. You have to go in through the low level listener interface. We could consider exposing the JobProgressListener directly - I think it's been factored nicely so it's fairly decoupled from the UI. The concern is this is a semi-internal piece of

Re: Largest Spark Cluster

2014-04-04 Thread Patrick Wendell
Hey Parviz, There was a similar thread a while ago... I think that many companies like to be discrete about the size of large clusters. But of course it would be great if people wanted to share openly :) For my part - I can say that Spark has been benchmarked on hundreds-of-nodes clusters before

Re: How to create a RPM package

2014-04-04 Thread Patrick Wendell
We might be able to incorporate the maven rpm plugin into our build. If that can be done in an elegant way it would be nice to have that distribution target for people who wanted to try this with arbitrary Spark versions... Personally I have no familiarity with that plug-in, so curious if anyone

Re: trouble with join on large RDDs

2014-04-07 Thread Patrick Wendell
On Mon, Apr 7, 2014 at 7:37 PM, Brad Miller bmill...@eecs.berkeley.eduwrote: I am running the latest version of PySpark branch-0.9 and having some trouble with join. One RDD is about 100G (25GB compressed and serialized in memory) with 130K records, the other RDD is about 10G (2.5G

Re: programmatic way to tell Spark version

2014-04-10 Thread Patrick Wendell
with the version. After futzing around a bit I realized it was printed out (quite conspicuously) in the shell startup banner. On Sat, Feb 22, 2014 at 7:15 PM, Patrick Wendell pwend...@gmail.comwrote: AFIAK - We don't have any way to do this right now. Maybe we could add a getVersion method

Re: programmatic way to tell Spark version

2014-04-10 Thread Patrick Wendell
...@realimpactanalytics.com *FR *+32 485 91 87 31 *| **Skype* pierre.borckmans On 10 Apr 2014, at 23:05, Patrick Wendell pwend...@gmail.com wrote: I think this was solved in a recent merge: https://github.com/apache/spark/pull/204/files#diff-364713d7776956cb8b0a771e9b62f82dR779 Is that what

Re: Spark on YARN performance

2014-04-11 Thread Patrick Wendell
To reiterate what Tom was saying - the code that runs inside of Spark on YARN is exactly the same code that runs in any deployment mode. There shouldn't be any performance difference once your application starts (assuming you are comparing apples-to-apples in terms of hardware). The differences

Re: running tests selectively

2014-04-20 Thread Patrick Wendell
I put some notes in this doc: https://cwiki.apache.org/confluence/display/SPARK/Useful+Developer+Tools On Sun, Apr 20, 2014 at 8:58 PM, Arun Ramakrishnan sinchronized.a...@gmail.com wrote: I would like to run some of the tests selectively. I am in branch-1.0 Tried the following two

Re: Task splitting among workers

2014-04-20 Thread Patrick Wendell
For a HadoopRDD, first the spark scheduler calculates the number of tasks based on input splits. Usually people use this with HDFS data so in that case it's based on HDFS blocks. If the HDFS datanodes are co-located with the Spark cluster then it will try to run the tasks on the data node that

Re: compile spark 0.9.1 in hadoop 2.2 above exception

2014-04-24 Thread Patrick Wendell
Try running sbt/sbt clean and re-compiling. Any luck? On Thu, Apr 24, 2014 at 5:33 PM, martin.ou martin...@orchestrallinc.cnwrote: occure exception when compile spark 0.9.1 using sbt,env: hadoop 2.3 1. SPARK_HADOOP_VERSION=2.3.0 SPARK_YARN=true sbt/sbt assembly 2.found Exception:

Re: pySpark memory usage

2014-04-28 Thread Patrick Wendell
Hey Jim, This IOException thing is a general issue that we need to fix and your observation is spot-in. There is actually a JIRA for it here I created a few days ago: https://issues.apache.org/jira/browse/SPARK-1579 Aaron is assigned on that one but not actively working on it, so we'd welcome a

Re: Running a spark-submit compatible app in spark-shell

2014-04-28 Thread Patrick Wendell
What about if you run ./bin/spark-shell --driver-class-path=/path/to/your/jar.jar I think either this or the --jars flag should work, but it's possible there is a bug with the --jars flag when calling the Repl. On Mon, Apr 28, 2014 at 4:30 PM, Roger Hoover roger.hoo...@gmail.comwrote: A

Re: launching concurrent jobs programmatically

2014-04-28 Thread Patrick Wendell
In general, as Andrew points out, it's possible to submit jobs from multiple threads and many Spark applications do this. One thing to check out is the job server from Ooyala, this is an application on top of Spark that has an automated submission API: https://github.com/ooyala/spark-jobserver

Re: How fast would you expect shuffle serialize to be?

2014-04-29 Thread Patrick Wendell
Is this the serialization throughput per task or the serialization throughput for all the tasks? On Tue, Apr 29, 2014 at 9:34 PM, Liu, Raymond raymond@intel.com wrote: Hi I am running a WordCount program which count words from HDFS, and I noticed that the serializer part of code

Re: JavaSparkConf

2014-04-29 Thread Patrick Wendell
This class was made to be java friendly so that we wouldn't have to use two versions. The class itself is simple. But I agree adding java setters would be nice. On Tue, Apr 29, 2014 at 8:32 PM, Soren Macbeth so...@yieldbot.com wrote: There is a JavaSparkContext, but no JavaSparkConf object. I

Re: Union of 2 RDD's only returns the first one

2014-04-29 Thread Patrick Wendell
order. RDD order is also what allows me to get the top k out of RDD by doing RDD.sort().take(). Am I misunderstanding it? Or, is it just when RDD is written to disk that the order is not well preserved? Thanks in advance! Mingyu On 1/22/14, 4:46 PM, Patrick Wendell pwend...@gmail.com wrote

Re: Reading multiple S3 objects, transforming, writing back one

2014-04-30 Thread Patrick Wendell
This is a consequence of the way the Hadoop files API works. However, you can (fairly easily) add code to just rename the file because it will always produce the same filename. (heavy use of pseudo code) dir = /some/dir rdd.coalesce(1).saveAsTextFile(dir) f = new File(dir + part-0)

Re: Reading multiple S3 objects, transforming, writing back one

2014-05-03 Thread Patrick Wendell
but not ideal for the longer term as data size increases. I understand it's a limitation of the Hadoop API but ultimately it must be possible to dump a RDD to a single S3 object :) On Wednesday, April 30, 2014 7:01 PM, Patrick Wendell pwend...@gmail.com wrote: This is a consequence of the way

Re: Setting the Scala version in the EC2 script?

2014-05-03 Thread Patrick Wendell
Spark will only work with Scala 2.10... are you trying to do a minor version upgrade or upgrade to a totally different version? You could do this as follows if you want: 1. Fork the spark-ec2 repository and change the file here: https://github.com/mesos/spark-ec2/blob/v2/scala/init.sh 2. Modify

Re: when to use broadcast variables

2014-05-03 Thread Patrick Wendell
Broadcast variables need to fit entirely in memory - so that's a pretty good litmus test for whether or not to broadcast a smaller dataset or turn it into an RDD. On Fri, May 2, 2014 at 7:50 AM, Prashant Sharma scrapco...@gmail.com wrote: I had like to be corrected on this but I am just trying

Re: spark ec2 error

2014-05-04 Thread Patrick Wendell
Hey Jeremy, This is actually a big problem - thanks for reporting it, I'm going to revert this change until we can make sure it is backwards compatible. - Patrick On Sun, May 4, 2014 at 2:00 PM, Jeremy Freeman freeman.jer...@gmail.com wrote: Hi all, A heads up in case others hit this and are

Re: spark ec2 error

2014-05-04 Thread Patrick Wendell
PM, Patrick Wendell pwend...@gmail.com wrote: Hey Jeremy, This is actually a big problem - thanks for reporting it, I'm going to revert this change until we can make sure it is backwards compatible. - Patrick On Sun, May 4, 2014 at 2:00 PM, Jeremy Freeman freeman.jer...@gmail.com wrote

Re: 1.0.0 Release Date?

2014-05-14 Thread Patrick Wendell
Hey Brian, We've had a fairly stable 1.0 branch for a while now. I've started voting on the dev list last night... voting can take some time but it usually wraps up anywhere from a few days to weeks. However, you can get started right now with the release candidates. These are likely to be

Re: little confused about SPARK_JAVA_OPTS alternatives

2014-05-15 Thread Patrick Wendell
Just wondering - how are you launching your application? If you want to set values like this the right way is to add them to the SparkConf when you create a SparkContext. val conf = new SparkConf().set(spark.akka.frameSize, 1).setAppName(...).setMaster(...) val sc = new SparkContext(conf) -

Announcing Spark 1.0.0

2014-05-30 Thread Patrick Wendell
I'm thrilled to announce the availability of Spark 1.0.0! Spark 1.0.0 is a milestone release as the first in the 1.0 line of releases, providing API stability for Spark's core interfaces. Spark 1.0.0 is Spark's largest release ever, with contributions from 117 developers. I'd like to thank

Re: Announcing Spark 1.0.0

2014-05-30 Thread Patrick Wendell
, Patrick Wendell pwend...@gmail.com mailto:pwend...@gmail.com wrote: I'm thrilled to announce the availability of Spark 1.0.0! Spark 1.0.0 is a milestone release as the first in the 1.0 line of releases, providing API stability for Spark's core interfaces. Spark 1.0.0 is Spark's

Re: Yay for 1.0.0! EC2 Still has problems.

2014-05-30 Thread Patrick Wendell
Hi Jeremy, That's interesting, I don't think anyone has ever reported an issue running these scripts due to Python incompatibility, but they may require Python 2.7+. I regularly run them from the AWS Ubuntu 12.04 AMI... that might be a good place to start. But if there is a straightforward way to

Re: Unable to execute saveAsTextFile on multi node mesos

2014-05-31 Thread Patrick Wendell
Can you look at the logs from the executor or in the UI? They should give an exception with the reason for the task failure. Also in the future, for this type of e-mail please only e-mail the user@ list and not both lists. - Patrick On Sat, May 31, 2014 at 3:22 AM, prabeesh k

Re: How can I dispose an Accumulator?

2014-05-31 Thread Patrick Wendell
Hey There, You can remove an accumulator by just letting it go out of scope and it will be garbage collected. For broadcast variables we actually store extra information for it, so we provide hooks for users to remove the associated state. There is no such need for accumulators, though. -

Re: Spark hook to create external process

2014-05-31 Thread Patrick Wendell
Currently, an executor is always run in it's own JVM, so it should be possible to just use some static initialization to e.g. launch a sub-process and set up a bridge with which to communicate. This is would be a fairly advanced use case, however. - Patrick On Thu, May 29, 2014 at 8:39 PM,

Re: possible typos in spark 1.0 documentation

2014-05-31 Thread Patrick Wendell
1. ctx is an instance of JavaSQLContext but the textFile method is called as a member of ctx. According to the API JavaSQLContext does not have such a member, so im guessing this should be sc instead. Yeah, I think you are correct. 2. In that same code example the object sqlCtx is

Re: getPreferredLocations

2014-05-31 Thread Patrick Wendell
1) Is there a guarantee that a partition will only be processed on a node which is in the getPreferredLocations set of nodes returned by the RDD ? No there isn't, by default Spark may schedule in a non preferred location after `spark.locality.wait` has expired.

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

2014-05-31 Thread Patrick Wendell
I think there are a few ways to do this... the simplest one might be to manually build a set of comma-separated paths that excludes the bad file, and pass that to textFile(). When you call textFile() under the hood it is going to pass your filename string to hadoopFile() which calls

Re: Yay for 1.0.0! EC2 Still has problems.

2014-06-01 Thread Patrick Wendell
Hey just to clarify this - my understanding is that the poster (Jeremey) was using a custom AMI to *launch* spark-ec2. I normally launch spark-ec2 from my laptop. And he was looking for an AMI that had a high enough version of python. Spark-ec2 itself has a flag -a that allows you to give a

Re: Using sbt-pack with Spark 1.0.0

2014-06-01 Thread Patrick Wendell
One potential issue here is that mesos is using classifiers now to publish there jars. It might be that sbt-pack has trouble with dependencies that are published using classifiers. I'm pretty sure mesos is the only dependency in Spark that is using classifiers, so that's why I mention it. On Sun,

Re: Using sbt-pack with Spark 1.0.0

2014-06-01 Thread Patrick Wendell
https://github.com/apache/spark/blob/master/project/SparkBuild.scala#L350 On Sun, Jun 1, 2014 at 11:03 AM, Patrick Wendell pwend...@gmail.com wrote: One potential issue here is that mesos is using classifiers now to publish there jars. It might be that sbt-pack has trouble with dependencies

Re: spark 1.0.0 on yarn

2014-06-01 Thread Patrick Wendell
.. -Simon On Sun, Jun 1, 2014 at 1:57 PM, Patrick Wendell pwend...@gmail.com wrote: I would agree with your guess, it looks like the yarn library isn't correctly finding your yarn-site.xml file. If you look in yarn-site.xml do you definitely the resource manager address/addresses? Also, you

Re: spark 1.0.0 on yarn

2014-06-02 Thread Patrick Wendell
. -Simon On Sun, Jun 1, 2014 at 9:03 PM, Patrick Wendell pwend...@gmail.com wrote: As a debugging step, does it work if you use a single resource manager with the key yarn.resourcemanager.address instead of using two named resource managers? I wonder if somehow the YARN client can't

Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Patrick Wendell
Hey There, The issue was that the old behavior could cause users to silently overwrite data, which is pretty bad, so to be conservative we decided to enforce the same checks that Hadoop does. This was documented by this JIRA: https://issues.apache.org/jira/browse/SPARK-1100

Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Patrick Wendell
-1677 is talking about the same thing? How about assigning it to me? I think I missed the configuration part in my previous commit, though I declared that in the PR description Best, -- Nan Zhu On Monday, June 2, 2014 at 3:03 PM, Patrick Wendell wrote: Hey There, The issue

Re: pyspark problems on yarn (job not parallelized, and Py4JJavaError)

2014-06-02 Thread Patrick Wendell
Are you building Spark with Java 6 or Java 7. Java 6 uses the extended Zip format and Java 7 uses Zip64. I think we've tried to add some build warnings if Java 7 is used, for this reason: https://github.com/apache/spark/blob/master/make-distribution.sh#L102 Any luck if you use JDK 6 to compile?

Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Patrick Wendell
, Jun 2, 2014 at 12:44 PM, Patrick Wendell pwend...@gmail.com wrote: Thanks for pointing that out. I've assigned you to SPARK-1677 (I think I accidentally assigned myself way back when I created it). This should be an easy fix. On Mon, Jun 2, 2014 at 12:19 PM, Nan Zhu zhunanmcg...@gmail.com

Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Patrick Wendell
, 2014 at 4:25 PM, Sean Owen so...@cloudera.com wrote: Is there a third way? Unless I miss something. Hadoop's OutputFormat wants the target dir to not exist no matter what, so it's just a question of whether Spark deletes it for you or errors. On Tue, Jun 3, 2014 at 12:22 AM, Patrick Wendell

Re: pyspark problems on yarn (job not parallelized, and Py4JJavaError)

2014-06-02 Thread Patrick Wendell
. The standard installation guide didn't say anything about java 7 and suggested to do -DskipTests for the build.. http://spark.apache.org/docs/latest/building-with-maven.html So, I didn't see the warning message... On Mon, Jun 2, 2014 at 3:48 PM, Patrick Wendell pwend...@gmail.com wrote

Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Patrick Wendell
, Jun 2, 2014 at 10:39 PM, Patrick Wendell pwend...@gmail.com wrote: (B) Semantics in Spark 1.0 and earlier: Do you mean 1.0 and later? Option (B) with the exception-on-clobber sounds fine to me, btw. My use pattern is probably common but not universal, and deleting user files is indeed

Re: spark 1.0 not using properties file from SPARK_CONF_DIR

2014-06-03 Thread Patrick Wendell
You can set an arbitrary properties file by adding --properties-file argument to spark-submit. It would be nice to have spark-submit also look in SPARK_CONF_DIR as well by default. If you opened a JIRA for that I'm sure someone would pick it up. On Tue, Jun 3, 2014 at 7:47 AM, Eugen Cepoi

Re: Spark 1.0.0 fails if mesos.coarse set to true

2014-06-04 Thread Patrick Wendell
Hey, thanks a lot for reporting this. Do you mind making a JIRA with the details so we can track it? - Patrick On Wed, Jun 4, 2014 at 9:24 AM, Marek Wiewiorka marek.wiewio...@gmail.com wrote: Exactly the same story - it used to work with 0.9.1 and does not work anymore with 1.0.0. I ran tests

Re: is there any easier way to define a custom RDD in Java

2014-06-04 Thread Patrick Wendell
Hey There, This is only possible in Scala right now. However, this is almost never needed since the core API is fairly flexible. I have the same question as Andrew... what are you trying to do with your RDD? - Patrick On Wed, Jun 4, 2014 at 7:49 AM, Andrew Ash and...@andrewash.com wrote: Just

Re: error with cdh 5 spark installation

2014-06-04 Thread Patrick Wendell
Hey Chirag, Those init scripts are part of the Cloudera Spark package (they are not in the Spark project itself) so you might try e-mailing their support lists directly. - Patrick On Wed, Jun 4, 2014 at 7:19 AM, chirag lakhani chirag.lakh...@gmail.com wrote: I recently spun up an AWS cluster

Re: Can't seem to link external/twitter classes from my own app

2014-06-04 Thread Patrick Wendell
Hey Jeremy, The issue is that you are using one of the external libraries and these aren't actually packaged with Spark on the cluster, so you need to create an uber jar that includes them. You can look at the example here (I recently did this for a kafka project and the idea is the same):

Re: Trouble launching EC2 Cluster with Spark

2014-06-04 Thread Patrick Wendell
Hey Sam, You mentioned two problems here, did your VPC error message get fixed or only the key permissions problem? I noticed we had some report a similar issue with the VPC stuff a long time back (but there is no real resolution here): https://spark-project.atlassian.net/browse/SPARK-1166 If

Re: Setting executor memory when using spark-shell

2014-06-06 Thread Patrick Wendell
In 1.0+ you can just pass the --executor-memory flag to ./bin/spark-shell. On Fri, Jun 6, 2014 at 12:32 AM, Oleg Proudnikov oleg.proudni...@gmail.com wrote: Thank you, Hassan! On 6 June 2014 03:23, hassan hellfire...@gmail.com wrote: just use -Dspark.executor.memory= -- View this

Re: Spark 1.0 embedded Hive libraries

2014-06-06 Thread Patrick Wendell
They are forked and slightly modified for two reasons: (a) Hive embeds a bunch of other dependencies in their published jars such that it makes it really hard for other projects to depend on them. If you look at the hive-exec jar they copy a bunch of other dependencies directly into this jar. We

Re: Strange problem with saveAsTextFile after upgrade Spark 0.9.0-1.0.0

2014-06-08 Thread Patrick Wendell
Paul, Could you give the version of Java that you are building with and the version of Java you are running with? Are they the same? Just off the cuff, I wonder if this is related to: https://issues.apache.org/jira/browse/SPARK-1520 If it is, it could appear that certain functions are not in

Re: Strange problem with saveAsTextFile after upgrade Spark 0.9.0-1.0.0

2014-06-08 Thread Patrick Wendell
Also I should add - thanks for taking time to help narrow this down! On Sun, Jun 8, 2014 at 1:02 PM, Patrick Wendell pwend...@gmail.com wrote: Paul, Could you give the version of Java that you are building with and the version of Java you are running with? Are they the same? Just off

Re: Strange problem with saveAsTextFile after upgrade Spark 0.9.0-1.0.0

2014-06-08 Thread Patrick Wendell
12:05 org/apache/spark/rdd/RDD$anonfun$saveAsTextFile$1.class 1560 06-08-14 12:05 org/apache/spark/rdd/RDD$anonfun$saveAsTextFile$2.class Best. -- Paul — p...@mult.ifario.us | Multifarious, Inc. | http://mult.ifario.us/ On Sun, Jun 8, 2014 at 1:02 PM, Patrick Wendell pwend

Re: Setting spark memory limit

2014-06-09 Thread Patrick Wendell
I you run locally then Spark doesn't launch remote executors. However, in this case you can set the memory with --spark-driver-memory flag to spark-submit. Does that work? - Patrick On Mon, Jun 9, 2014 at 3:24 PM, Henggang Cui cuihengg...@gmail.com wrote: Hi, I'm trying to run the SimpleApp

Re: Enormous EC2 price jump makes r3.large patch more important

2014-06-17 Thread Patrick Wendell
Hey Jeremy, This is patched in the 1.0 and 0.9 branches of Spark. We're likely to make a 1.0.1 release soon (this patch being one of the main reasons), but if you are itching for this sooner, you can just checkout the head of branch-1.0 and you will be able to use r3.XXX instances. - Patrick On

Re: Enormous EC2 price jump makes r3.large patch more important

2014-06-17 Thread Patrick Wendell
By the way, in case it's not clear, I mean our maintenance branches: https://github.com/apache/spark/tree/branch-1.0 On Tue, Jun 17, 2014 at 8:35 PM, Patrick Wendell pwend...@gmail.com wrote: Hey Jeremy, This is patched in the 1.0 and 0.9 branches of Spark. We're likely to make a 1.0.1

Re: Enormous EC2 price jump makes r3.large patch more important

2014-06-17 Thread Patrick Wendell
to lean in the direction of Cassandra as the distributed data store... On Wed, Jun 18, 2014 at 1:46 PM, Patrick Wendell pwend...@gmail.com wrote: By the way, in case it's not clear, I mean our maintenance branches: https://github.com/apache/spark/tree/branch-1.0 On Tue, Jun 17, 2014 at 8:35

Re: Wildcard support in input path

2014-06-17 Thread Patrick Wendell
These paths get passed directly to the Hadoop FileSystem API and I think the support globbing out-of-the box. So AFAIK it should just work. On Tue, Jun 17, 2014 at 9:09 PM, MEETHU MATHEW meethu2...@yahoo.co.in wrote: Hi Jianshi, I have used wild card characters (*) in my program and it

Re: Java IO Stream Corrupted - Invalid Type AC?

2014-06-17 Thread Patrick Wendell
Out of curiosity - are you guys using speculation, shuffle consolidation, or any other non-default option? If so that would help narrow down what's causing this corruption. On Tue, Jun 17, 2014 at 10:40 AM, Surendranauth Hiraman suren.hira...@velos.io wrote: Matt/Ryan, Did you make any headway

Re: Trailing Tasks Saving to HDFS

2014-06-19 Thread Patrick Wendell
I'll make a comment on the JIRA - thanks for reporting this, let's get to the bottom of it. On Thu, Jun 19, 2014 at 11:19 AM, Surendranauth Hiraman suren.hira...@velos.io wrote: I've created an issue for this but if anyone has any advice, please let me know. Basically, on about 10 GBs of

Re: 1.0.1 release plan

2014-06-20 Thread Patrick Wendell
Hey There, I'd like to start voting on this release shortly because there are a few important fixes that have queued up. We're just waiting to fix an akka issue. I'd guess we'll cut a vote in the next few days. - Patrick On Thu, Jun 19, 2014 at 10:47 AM, Mingyu Kim m...@palantir.com wrote: Hi

Re: hadoop + yarn + spark

2014-06-27 Thread Patrick Wendell
Hi There, There is an issue with PySpark-on-YARN that requires users build with Java 6. The issue has to do with how Java 6 and 7 package jar files differently. Can you try building spark with Java 6 and trying again? - Patrick On Fri, Jun 27, 2014 at 5:00 PM, sdeb sangha...@gmail.com wrote:

Re: How to clear the list of Completed Appliations in Spark web UI?

2014-07-09 Thread Patrick Wendell
There isn't currently a way to do this, but it will start dropping older applications once more than 200 are stored. On Wed, Jul 9, 2014 at 4:04 PM, Haopu Wang hw...@qilinsoft.com wrote: Besides restarting the Master, is there any other way to clear the Completed Applications in Master web UI?

Re: Purpose of spark-submit?

2014-07-09 Thread Patrick Wendell
It fulfills a few different functions. The main one is giving users a way to inject Spark as a runtime dependency separately from their program and make sure they get exactly the right version of Spark. So a user can bundle an application and then use spark-submit to send it to different types of

Re: issues with ./bin/spark-shell for standalone mode

2014-07-09 Thread Patrick Wendell
Hey Mikhail, I think (hope?) the -em and -dm options were never in an official Spark release. They were just in the master branch at some point. Did you use these during a previous Spark release or were you just on master? - Patrick On Wed, Jul 9, 2014 at 9:18 AM, Mikhail Strebkov

Announcing Spark 1.0.1

2014-07-11 Thread Patrick Wendell
I am happy to announce the availability of Spark 1.0.1! This release includes contributions from 70 developers. Spark 1.0.0 includes fixes across several areas of Spark, including the core API, PySpark, and MLlib. It also includes new features in Spark's (alpha) SQL library, including support for

Re: Catalyst dependency on Spark Core

2014-07-14 Thread Patrick Wendell
Adding new build modules is pretty high overhead, so if this is a case where a small amount of duplicated code could get rid of the dependency, that could also be a good short-term option. - Patrick On Mon, Jul 14, 2014 at 2:15 PM, Matei Zaharia matei.zaha...@gmail.com wrote: Yeah, I'd just add

Re: how to publish spark inhouse?

2014-07-28 Thread Patrick Wendell
All of the scripts we use to publish Spark releases are in the Spark repo itself, so you could follow these as a guideline. The publishing process in Maven is similar to in SBT: https://github.com/apache/spark/blob/master/dev/create-release/create-release.sh#L65 On Mon, Jul 28, 2014 at 12:39 PM,

Re: Compiling Spark master (284771ef) with sbt/sbt assembly fails on EC2

2014-08-01 Thread Patrick Wendell
This is a Scala bug - I filed something upstream, hopefully they can fix it soon and/or we can provide a work around: https://issues.scala-lang.org/browse/SI-8772 - Patrick On Fri, Aug 1, 2014 at 3:15 PM, Holden Karau hol...@pigscanfly.ca wrote: Currently scala 2.10.2 can't be pulled in from

Re: Compiling Spark master (284771ef) with sbt/sbt assembly fails on EC2

2014-08-01 Thread Patrick Wendell
I've had intermiddent access to the artifacts themselves, but for me the directory listing always 404's. I think if sbt hits a 404 on the directory, it sends a somewhat confusing error message that it can't download the artifact. - Patrick On Fri, Aug 1, 2014 at 3:28 PM, Shivaram Venkataraman

Re: disable log4j for spark-shell

2014-08-03 Thread Patrick Wendell
If you want to customize the logging behavior - the simplest way is to copy conf/log4j.properties.tempate to conf/log4j.properties. Then you can go and modify the log level in there. The spark shells should pick this up. On Sun, Aug 3, 2014 at 6:16 AM, Sean Owen so...@cloudera.com wrote:

Re: What should happen if we try to cache more data than the cluster can hold in memory?

2014-08-04 Thread Patrick Wendell
It seems possible that you are running out of memory unrolling a single partition of the RDD. This is something that can cause your executor to OOM, especially if the cache is close to being full so the executor doesn't have much free memory left. How large are your executors? At the time of

Re: What should happen if we try to cache more data than the cluster can hold in memory?

2014-08-04 Thread Patrick Wendell
BTW - the reason why the workaround could help is because when persisting to DISK_ONLY, we explicitly avoid materializing the RDD partition in memory... we just pass it through to disk On Mon, Aug 4, 2014 at 1:10 AM, Patrick Wendell pwend...@gmail.com wrote: It seems possible that you

Re: Issues with HDP 2.4.0.2.1.3.0-563

2014-08-04 Thread Patrick Wendell
For hortonworks, I believe it should work to just link against the corresponding upstream version. I.e. just set the Hadoop version to 2.4.0 Does that work? - Patrick On Mon, Aug 4, 2014 at 12:13 AM, Ron's Yahoo! zlgonza...@yahoo.com.invalid wrote: Hi, Not sure whose issue this is, but if

Re: Cached RDD Block Size - Uneven Distribution

2014-08-04 Thread Patrick Wendell
Are you directly caching files from Hadoop or are you doing some transformation on them first? If you are doing a groupBy or some type of transformation, then you could be causing data skew that way. On Sun, Aug 3, 2014 at 1:19 PM, iramaraju iramar...@gmail.com wrote: I am running spark 1.0.0,

Re: Understanding RDD.GroupBy OutOfMemory Exceptions

2014-08-05 Thread Patrick Wendell
out sequentially on disk on one big file, you can call `sortByKey` with a hashed suffix as well. The sort functions are externalized in Spark 1.1 (which is in pre-release). - Patrick On Tue, Aug 5, 2014 at 2:39 PM, Jens Kristian Geyti sp...@jkg.dk wrote: Patrick Wendell wrote In the latest

Re: Advantage of using cache()

2014-08-20 Thread Patrick Wendell
Your rdd2 and rdd3 differ in two ways so it's hard to track the exact effect of caching. In rdd3, in addition to the fact that rdd will be cached, you are also doing a bunch of extra random number generation. So it will be hard to isolate the effect of caching. On Wed, Aug 20, 2014 at 7:48 AM,

  1   2   >