Re: Wish for 1.4: upper bound on # tasks in Mesos

2015-05-19 Thread Matei Zaharia
Hey Tom, Are you using the fine-grained or coarse-grained scheduler? For the coarse-grained scheduler, there is a spark.cores.max config setting that will limit the total # of cores it grabs. This was there in earlier versions too. Matei > On May 19, 2015, at 12:39 PM, Thomas Dudziak wrote: >

Re: Wish for 1.4: upper bound on # tasks in Mesos

2015-05-19 Thread Matei Zaharia
ome the limit of tasks per job :) > > cheers, > Tom > > On Tue, May 19, 2015 at 10:05 AM, Matei Zaharia <mailto:matei.zaha...@gmail.com>> wrote: > Hey Tom, > > Are you using the fine-grained or coarse-grained scheduler? For the > coarse-grained scheduler, there

Re: Spark logo license

2015-05-19 Thread Matei Zaharia
Check out Apache's trademark guidelines here: http://www.apache.org/foundation/marks/ Matei > On May 20, 2015, at 12:02 AM, Justin Pihony wrote: > > What is the license on using the spark logo. Is it free to be used for > displaying commercially? > >

Re: map - reduce only with disk

2015-06-01 Thread Matei Zaharia
As long as you don't use cache(), these operations will go from disk to disk, and will only use a fixed amount of memory to build some intermediate results. However, note that because you're using groupByKey, that needs the values for each key to all fit in memory at once. In this case, if you'r

Re: map - reduce only with disk

2015-06-02 Thread Matei Zaharia
quot;spark.executor.memory", "115g") > conf.set("spark.shuffle.file.buffer.kb", "1000") > > my spark-env.sh: > ulimit -n 200000 > SPARK_JAVA_OPTS="-Xss1g -Xmx129g -d64 -XX:-UseGCOverheadLimit > -XX:-UseCompressedOops" > SPARK

Re: map - reduce only with disk

2015-06-02 Thread Matei Zaharia
like they do now? > > Thank you! > > 2015-06-02 21:25 GMT+02:00 Matei Zaharia <mailto:matei.zaha...@gmail.com>>: > You shouldn't have to persist the RDD at all, just call flatMap and reduce on > it directly. If you try to persist it, that will try to load the origin

Re: Equivalent to Storm's 'field grouping' in Spark.

2015-06-03 Thread Matei Zaharia
This happens automatically when you use the byKey operations, e.g. reduceByKey, updateStateByKey, etc. Spark Streaming keeps the state for a given set of keys on a specific node and sends new tuples with that key to that. Matei > On Jun 3, 2015, at 6:31 AM, allonsy wrote: > > Hi everybody, >

Re: Spark or Storm

2015-06-17 Thread Matei Zaharia
This documentation is only for writes to an external system, but all the counting you do within your streaming app (e.g. if you use reduceByKeyAndWindow to keep track of a running count) is exactly-once. When you write to a storage system, no matter which streaming framework you use, you'll have

Re: Spark or Storm

2015-06-17 Thread Matei Zaharia
derstand that > 1) There is no global ordering; e.g. an output operation for batch consisting > of offset [4,5,6] can be invoked before the operation for offset [1,2,3] > 2) If you wanted to achieve something similar to what TridentState does, > you'll have to do it yourself (for

Re: work around Size exceeds Integer.MAX_VALUE

2015-07-09 Thread Matei Zaharia
Thus means that one of your cached RDD partitions is bigger than 2 GB of data. You can fix it by having more partitions. If you read data from a file system like HDFS or S3, set the number of partitions higher in the sc.textFile, hadoopFile, etc methods (it's an optional second parameter to thos

Re: Switch RDD-based MLlib APIs to maintenance mode in Spark 2.0

2016-04-05 Thread Matei Zaharia
This sounds good to me as well. The one thing we should pay attention to is how we update the docs so that people know to start with the spark.ml classes. Right now the docs list spark.mllib first and also seem more comprehensive in that area than in spark.ml, so maybe people naturally move towa

Re: Apache Spark Slack

2016-05-16 Thread Matei Zaharia
I don't think any of the developers use this as an official channel, but all the ASF IRC channels are indeed on FreeNode. If there's demand for it, we can document this on the website and say that it's mostly for users to find other users. Development discussions should happen on the dev mailing

Updated Spark logo

2016-06-10 Thread Matei Zaharia
Hi all, FYI, we've recently updated the Spark logo at https://spark.apache.org/ to say "Apache Spark" instead of just "Spark". Many ASF projects have been doing this recently to make it clearer that they are associated with the ASF, and indeed the ASF's branding guidelines generally require that

Re: The Future Of DStream

2016-07-27 Thread Matei Zaharia
Yup, they will definitely coexist. Structured Streaming is currently alpha and will probably be complete in the next few releases, but Spark Streaming will continue to exist, because it gives the user more low-level control. It's similar to DataFrames vs RDDs (RDDs are the lower-level API for wh

Re: Dropping late date in Structured Streaming

2016-08-06 Thread Matei Zaharia
Yes, a built-in mechanism is planned in future releases. You can also drop it using a filter for now but the stateful operators will still keep state for old windows. Matei > On Aug 6, 2016, at 9:40 AM, Amit Sela wrote: > > I've noticed that when using Structured Streaming with event-time win

Re: unsubscribe

2016-08-10 Thread Matei Zaharia
To unsubscribe, please send an email to user-unsubscr...@spark.apache.org from the address you're subscribed from. Matei > On Aug 10, 2016, at 12:48 PM, Sohil Jain wrote: > > - To unsubscribe e-mail: user-unsubscr...@spark.

Re: Read from AWS s3 with out having to hard-code sensitive keys

2016-01-11 Thread Matei Zaharia
In production, I'd recommend using IAM roles to avoid having keys altogether. Take a look at http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/iam-roles-for-amazon-ec2.html. Matei > On Jan 11, 2016, at 11:32 AM, Sabarish Sasidharan > wrote: > > If you are on EMR, these can go into your hdfs

Re: simultaneous actions

2016-01-15 Thread Matei Zaharia
RDDs actually are thread-safe, and quite a few applications use them this way, e.g. the JDBC server. Matei > On Jan 15, 2016, at 2:10 PM, Jakob Odersky wrote: > > I don't think RDDs are threadsafe. > More fundamentally however, why would you want to run RDD actions in > parallel? The idea beh

Re: Compiling only MLlib?

2016-01-15 Thread Matei Zaharia
Have you tried just downloading a pre-built package, or linking to Spark through Maven? You don't need to build it unless you are changing code inside it. Check out http://spark.apache.org/docs/latest/quick-start.html#self-contained-applications for how to link to it. Matei > On Jan 15, 2016,

Re: simultaneous actions

2016-01-17 Thread Matei Zaharia
rkers-become-available basis)? > > On 15 January 2016 at 11:44, Koert Kuipers <mailto:ko...@tresata.com>> wrote: > we run multiple actions on the same (cached) rdd all the time, i guess in > different threads indeed (its in akka) > > On Fri, Jan 15, 2016 at 2:40 PM,

Re: How to compile Spark with customized Hadoop?

2015-10-09 Thread Matei Zaharia
You can publish your version of Hadoop to your Maven cache with mvn publish (just give it a different version number, e.g. 2.7.0a) and then pass that as the Hadoop version to Spark's build (see http://spark.apache.org/docs/latest/building-spark.html

Re: Ranger-like Security on Spark

2015-09-03 Thread Matei Zaharia
If you run on YARN, you can use Kerberos, be authenticated as the right user, etc in the same way as MapReduce jobs. Matei > On Sep 3, 2015, at 1:37 PM, Daniel Schulz > wrote: > > Hi, > > I really enjoy using Spark. An obstacle to sell it to our clients currently > is the missing Kerberos-l

Re: Ranger-like Security on Spark

2015-09-03 Thread Matei Zaharia
policies as well? > > Best regards, Daniel. > > > On 03 Sep 2015, at 21:16, Matei Zaharia > <mailto:matei.zaha...@gmail.com>> wrote: > > > > If you run on YARN, you can use Kerberos, be authenticated as the right > > user, etc in the same way as MapRedu

Re: Is "spark streaming" streaming or mini-batch?

2016-08-23 Thread Matei Zaharia
I think people explained this pretty well, but in practice, this distinction is also somewhat of a marketing term, because every system will perform some kind of batching. For example, every time you use TCP, the OS and network stack may buffer multiple messages together and send them at once; a

Re: RESTful Endpoint and Spark

2016-10-06 Thread Matei Zaharia
This is exactly what the Spark SQL Thrift server does, if you just want to access it using JDBC. Matei > On Oct 6, 2016, at 4:27 PM, Benjamin Kim wrote: > > Has anyone tried to integrate Spark with a server farm of RESTful API > endpoints or even HTTP web-servers for that matter? I know it’s

Re: Structured Streaming with Kafka Source, does it work??

2016-11-06 Thread Matei Zaharia
The Kafka source will only appear in 2.0.2 -- see this thread for the current release candidate: https://lists.apache.org/thread.html/597d630135e9eb3ede54bb0cc0b61a2b57b189588f269a64b58c9243@%3Cdev.spark.apache.org%3E . You can try that right now if you want from the staging Maven repo shown th

Re: real world spark code

2017-07-25 Thread Matei Zaharia
You can also find a lot of GitHub repos for external packages here: http://spark.apache.org/third-party-projects.html Matei > On Jul 25, 2017, at 5:30 PM, Frank Austin Nothaft > wrote: > > There’s a number of real-world open source Spark applications in the sciences: > > genomics: > > githu

Re: Kill Spark Streaming JOB from Spark UI or Yarn

2017-08-27 Thread Matei Zaharia
The batches should all have the same application ID, so use that one. You can also find the application in the YARN UI to terminate it from there. Matei > On Aug 27, 2017, at 10:27 AM, KhajaAsmath Mohammed > wrote: > > Hi, > > I am new to spark streaming and not able to find an option to kil

Re: Spark 1.x - End of life

2017-10-19 Thread Matei Zaharia
Hi Ismael, It depends on what you mean by “support”. In general, there won’t be new feature releases for 1.X (e.g. Spark 1.7) because all the new features are being added to the master branch. However, there is always room for bug fix releases if there is a catastrophic bug, and committers can

Re: how can I run spark job in my environment which is a single Ubuntu host with no hadoop installed

2018-06-17 Thread Matei Zaharia
Maybe your application is overriding the master variable when it creates its SparkContext. I see you are still passing “yarn-client” as an argument later to it in your command. > On Jun 17, 2018, at 11:53 AM, Raymond Xie wrote: > > Thank you Subhash. > > Here is the new command: > spark-submi

Re: Is there any open source framework that converts Cypher to SparkSQL?

2018-09-16 Thread Matei Zaharia
GraphFrames (https://graphframes.github.io) offers a Cypher-like syntax that then executes on Spark SQL. > On Sep 14, 2018, at 2:42 AM, kant kodali wrote: > > Hi All, > > Is there any open source framework that converts Cypher to SparkSQL? > > Thanks! ---

Re: Should python-2 be supported in Spark 3.0?

2018-09-17 Thread Matei Zaharia
I’d like to understand the maintenance burden of Python 2 before deprecating it. Since it is not EOL yet, it might make sense to only deprecate it once it’s EOL (which is still over a year from now). Supporting Python 2+3 seems less burdensome than supporting, say, multiple Scala versions in the

Re: Should python-2 be supported in Spark 3.0?

2018-09-17 Thread Matei Zaharia
x27;t users prefer to get that notification sooner > rather than later? > > On Mon, Sep 17, 2018 at 12:58 PM Matei Zaharia > wrote: > I’d like to understand the maintenance burden of Python 2 before deprecating > it. Since it is not EOL yet, it might make sense to only deprecate it o

Re: Spark 2.4.0 artifact in Maven repository

2018-11-06 Thread Matei Zaharia
Hi Bartosz, This is because the vote on 2.4 has passed (you can see the vote thread on the dev mailing list) and we are just working to get the release into various channels (Maven, PyPI, etc), which can take some time. Expect to see an announcement soon once that’s done. Matei > On Nov 4, 20

Re: Why Apache Spark doesn't use Calcite?

2020-01-13 Thread Matei Zaharia
I’m pretty sure that Catalyst was built before Calcite, or at least in parallel. Calcite 1.0 was only released in 2015. From a technical standpoint, building Catalyst in Scala also made it more concise and easier to extend than an optimizer written in Java (you can find various presentations abo

Re: Why must the dstream.foreachRDD(...) parameter be serializable?

2015-01-27 Thread Matei Zaharia
I believe this is needed for driver recovery in Spark Streaming. If your Spark driver program crashes, Spark Streaming can recover the application by reading the set of DStreams and output operations from a checkpoint file (see https://spark.apache.org/docs/latest/streaming-programming-guide.htm

Re: Beginner in Spark

2015-02-06 Thread Matei Zaharia
You don't need HDFS or virtual machines to run Spark. You can just download it, unzip it and run it on your laptop. See http://spark.apache.org/docs/latest/index.html . Matei > On Feb 6, 2015, at 2:58 PM, David Fallside wrote: > > King, consid

Re: Berlin Apache Spark Meetup

2015-02-17 Thread Matei Zaharia
Thanks! I've added you. Matei > On Feb 17, 2015, at 4:06 PM, Ralph Bergmann | the4thFloor.eu > wrote: > > Hi, > > > there is a small Spark Meetup group in Berlin, Germany :-) > http://www.meetup.com/Berlin-Apache-Spark-Meetup/ > > Plaes add this group to the Meetups list at > https://spark.

Re: Querying JSON in Spark SQL

2015-03-16 Thread Matei Zaharia
The programming guide has a short example: http://spark.apache.org/docs/latest/sql-programming-guide.html#json-datasets . Note that once you infer a schema for a JSON dataset, you can also use nested path notation (e.

Re: IPyhon notebook command for spark need to be updated?

2015-03-20 Thread Matei Zaharia
Feel free to send a pull request to fix the doc (or say which versions it's needed in). Matei > On Mar 20, 2015, at 6:49 PM, Krishna Sankar wrote: > > Yep the command-option is gone. No big deal, just add the '%pylab inline' > command as part of your notebook. > Cheers > > > On Fri, Mar 20,

Re: Dataset announcement

2015-04-15 Thread Matei Zaharia
Very neat, Olivier; thanks for sharing this. Matei > On Apr 15, 2015, at 5:58 PM, Olivier Chapelle wrote: > > Dear Spark users, > > I would like to draw your attention to a dataset that we recently released, > which is as of now the largest machine learning dataset ever released; see > the fol

Re: Spark on Windows

2015-04-16 Thread Matei Zaharia
You could build Spark with Scala 2.11 on Mac / Linux and transfer it over to Windows. AFAIK it should build on Windows too, the only problem is that Maven might take a long time to download dependencies. What errors are you seeing? Matei > On Apr 16, 2015, at 9:23 AM, Arun Lists wrote: > > We

Re: large volume spark job spends most of the time in AppendOnlyMap.changeValue

2015-05-12 Thread Matei Zaharia
It could also be that your hash function is expensive. What is the key class you have for the reduceByKey / groupByKey? Matei > On May 12, 2015, at 10:08 AM, Night Wolf wrote: > > I'm seeing a similar thing with a slightly different stack trace. Ideas? > > org.apache.spark.util.collection.App

Re: SPARKTA: a real-time aggregation engine based on Spark Streaming

2015-05-14 Thread Matei Zaharia
...This is madness! > On May 14, 2015, at 9:31 AM, dmoralesdf wrote: > > Hi there, > > We have released our real-time aggregation engine based on Spark Streaming. > > SPARKTA is fully open source (Apache2) > > > You can checkout the slides showed up at the Strata past week: > > http://www.s

Re: SPARKTA: a real-time aggregation engine based on Spark Streaming

2015-05-14 Thread Matei Zaharia
(Sorry, for non-English people: that means it's a good thing.) Matei > On May 14, 2015, at 10:53 AM, Matei Zaharia wrote: > > ...This is madness! > >> On May 14, 2015, at 9:31 AM, dmoralesdf wrote: >> >> Hi there, >> >> We have released

Re: SecurityException when running tests with Spark 1.0.0

2014-06-02 Thread Matei Zaharia
You can just use the Maven build for now, even for Spark 1.0.0. Matei On Jun 2, 2014, at 5:30 PM, Mohit Nayak wrote: > Hey, > Yup that fixed it. Thanks so much! > > Is this the only solution, or could this be resolved in future versions of > Spark ? > > > On Mon, Jun 2, 2014 at 5:14 PM, Se

Re: mounting SSD devices of EC2 r3.8xlarge instances

2014-06-03 Thread Matei Zaharia
Those instance types are not yet supported by the scripts, but https://issues.apache.org/jira/browse/SPARK-1790 is tracking this issue and it will soon be fixed in both branch-0.9 and 1.0. The problem is that those drives are not formatted on r3 machines, whereas they are on the other instance t

Re: wholeTextFiles() : java.lang.IncompatibleClassChangeError: Found class org.apache.hadoop.mapreduce.TaskAttemptContext, but interface was expected

2014-06-03 Thread Matei Zaharia
Yeah unfortunately Hadoop 2 requires these binaries on Windows. Hadoop 1 runs just fine without them. Matei On Jun 3, 2014, at 10:33 AM, Sean Owen wrote: > I'd try the internet / SO first -- these are actually generic > Hadoop-related issues. Here I think you don't have HADOOP_HOME or > simila

Re: Better line number hints for logging?

2014-06-03 Thread Matei Zaharia
You can use RDD.setName to give it a name. There’s also a creationSite field that is private[spark] — we may want to add a public setter for that later. If the name isn’t enough and you’d like this, please open a JIRA issue for it. Matei On Jun 3, 2014, at 5:22 PM, John Salvatier wrote: > I h

Re: Invalid Class Exception

2014-06-03 Thread Matei Zaharia
What Java version do you have, and how did you get Spark (did you build it yourself by any chance or download a pre-built one)? If you build Spark yourself you need to do it with Java 6 — it’s a known issue because of the way Java 6 and 7 package JAR files. But I haven’t seen it result in this p

Re: Yay for 1.0.0! EC2 Still has problems.

2014-06-03 Thread Matei Zaharia
urce > on my "cluster controller". OK, I never liked that Amazon Linux AMI anyway. > I'm going to start from scratch again with an Ubuntu 12.04 instance, > hopefully that will be more auspicious... > > Meanwhile I'm learning scala... Great Turing's Ghost, it's th

Re: Upgradation to Spark 1.0.0

2014-06-03 Thread Matei Zaharia
You can copy your configuration from the old one. I’d suggest just downloading it to a different location on each node first for testing, then you can delete the old one if things work. On Jun 3, 2014, at 12:38 AM, MEETHU MATHEW wrote: > Hi , > > I am currently using SPARK 0.9 configured wit

Re: KMeans.train() throws NotSerializableException

2014-06-03 Thread Matei Zaharia
How is your RDD created? It might mean that something used in the process of creating it was not serializable. Matei On Jun 3, 2014, at 10:11 PM, bluejoe2008 wrote: > when i called KMeans.train(), an error happened: > > 14/06/04 13:02:29 INFO scheduler.DAGScheduler: Submitting Stage 3 > (Ma

Re: Join : Giving incorrect result

2014-06-04 Thread Matei Zaharia
If this isn’t the problem, it would be great if you can post the code for the program. Matei On Jun 4, 2014, at 12:58 PM, Xu (Simon) Chen wrote: > Maybe your two workers have different assembly jar files? > > I just ran into a similar problem that my spark-shell is using a different > jar fi

Re: reuse hadoop code in Spark

2014-06-04 Thread Matei Zaharia
Yes, you can write some glue in Spark to call these. Some functions to look at: - SparkContext.hadoopRDD lets you create an input RDD from an existing JobConf configured by Hadoop (including InputFormat, paths, etc) - RDD.mapPartitions lets you operate in all the values on one partition (block)

Re: Better line number hints for logging?

2014-06-04 Thread Matei Zaharia
line? (Of course you would have to click to expand it.) > > > On Wed, Jun 4, 2014 at 2:38 AM, John Salvatier wrote: > Ok, I will probably open a Jira. > > > On Tue, Jun 3, 2014 at 5:29 PM, Matei Zaharia wrote: > You can use RDD.setName to give it a name. There’s als

Re: pyspark join crash

2014-06-04 Thread Matei Zaharia
In PySpark, the data processed by each reduce task needs to fit in memory within the Python process, so you should use more tasks to process this dataset. Data is spilled to disk across tasks. I’ve created https://issues.apache.org/jira/browse/SPARK-2021 to track this — it’s something we’ve bee

Re: How can I dispose an Accumulator?

2014-06-04 Thread Matei Zaharia
All of these are disposed of automatically if you stop the context or exit the program. Matei On Jun 4, 2014, at 2:22 PM, Daniel Siegmann wrote: > Will the broadcast variables be disposed automatically if the context is > stopped, or do I still need to unpersist()? > > > On Sat, May 31, 20

Re: error loading large files in PySpark 0.9.0

2014-06-04 Thread Matei Zaharia
Ah, good to know! By the way in master we now have saveAsPickleFile (https://github.com/apache/spark/pull/755), and Nick Pentreath has been working on Hadoop InputFormats: https://github.com/apache/spark/pull/455. Would be good to have your input on both of those if you have a chance to try the

Re: pyspark join crash

2014-06-04 Thread Matei Zaharia
erhaps similar to the problem with python broadcast variables chewing > through memory https://spark-project.atlassian.net/browse/SPARK-1065). > > -Brad > > > > On Wed, Jun 4, 2014 at 1:42 PM, Matei Zaharia wrote: > In PySpark, the data processed by each reduce task

Re: Why Scala?

2014-06-04 Thread Matei Zaharia
n APIs in Spark Streaming? > Anytime frame on this? > > Thanks! > > John > > > On Thu, May 29, 2014 at 4:19 PM, Matei Zaharia > wrote: > Quite a few people ask this question and the answer is pretty simple. When we > started Spark, we had two goals — we want

Re: Logistic Regression MLLib Slow

2014-06-04 Thread Matei Zaharia
Are you using the logistic_regression.py in examples/src/main/python or examples/src/main/python/mllib? The first one is an example of writing logistic regression by hand and won’t be as efficient as the MLlib one. I suggest trying the MLlib one. You may also want to check how many iterations i

Re: Logistic Regression MLLib Slow

2014-06-04 Thread Matei Zaharia
t; The MLLib version of logistic regression doesn't seem to use all the cores on > my machine. > > Regards, > Krishna > > > > On Wed, Jun 4, 2014 at 6:47 PM, Matei Zaharia wrote: > Are you using the logistic_regression.py in examples/src/main/python or > exam

Re: reuse hadoop code in Spark

2014-06-05 Thread Matei Zaharia
tility java classes I wrote. Can I reuse the map function in java and port > it into Spark? > > Best regards, > Wei > > > - > Wei Tan, PhD > Research Staff Member > IBM T. J. Watson Research Center > http://researcher.

Re: Join : Giving incorrect result

2014-06-05 Thread Matei Zaharia
when memory is not enough. > > Thanks Chen for your observation. I get this problem on single worker so > there will not be any mismatch of jars. On two workers, since executor memory > gets doubled the code works fine. > > Regards, > Ajay > > > On Thursday,

Re: How to enable fault-tolerance?

2014-06-09 Thread Matei Zaharia
If this is a useful feature for local mode, we should open a JIRA to document the setting or improve it (I’d prefer to add a spark.local.retries property instead of a special URL format). We initially disabled it for everything except unit tests because 90% of the time an exception in local mode

Re: Is spark context in local mode thread-safe?

2014-06-09 Thread Matei Zaharia
You currently can’t have multiple SparkContext objects in the same JVM, but within a SparkContext, all of the APIs are thread-safe so you can share that context between multiple threads. The other issue you’ll run into is that in each thread where you want to use Spark, you need to use SparkEnv.

Re: Is spark context in local mode thread-safe?

2014-06-09 Thread Matei Zaharia
ncerely, > > DB Tsai > --- > My Blog: https://www.dbtsai.com > LinkedIn: https://www.linkedin.com/in/dbtsai > > > On Mon, Jun 9, 2014 at 4:34 PM, Matei Zaharia wrote: >> You currently can’t have multiple SparkContext obj

Re: Is spark context in local mode thread-safe?

2014-06-09 Thread Matei Zaharia
bug tomcat code. If we can > disable the UI http Server; it would be much simpler to handle than having > two http containers to deal with. > > Chester > > > > On Monday, June 9, 2014 4:35 PM, Matei Zaharia > wrote: > > > You currently can’t have multiple Spar

Re: How to specify executor memory in EC2 ?

2014-06-10 Thread Matei Zaharia
It might be that conf/spark-env.sh on EC2 is configured to set it to 512, and is overriding the application’s settings. Take a look in there and delete that line if possible. Matei On Jun 10, 2014, at 2:38 PM, Aliaksei Litouka wrote: > I am testing my application in EC2 cluster of m3.medium

Re: Compression with DISK_ONLY persistence

2014-06-11 Thread Matei Zaharia
Yes, actually even if you don’t set it to true, on-disk data is compressed. (This setting only affects serialized data in memory). Matei On Jun 11, 2014, at 2:56 PM, Surendranauth Hiraman wrote: > Hi, > > Will spark.rdd.compress=true enable compression when using DISK_ONLY > persistence? >

Re: Powered by Spark addition

2014-06-11 Thread Matei Zaharia
Alright, added you. Matei On Jun 11, 2014, at 1:28 PM, Derek Mansen wrote: > Hello, I was wondering if we could add our organization to the "Powered by > Spark" page. The information is: > > Name: Vistar Media > URL: www.vistarmedia.com > Description: Location technology company enabling bran

Re: When to use CombineByKey vs reduceByKey?

2014-06-11 Thread Matei Zaharia
combineByKey is designed for when your return type from the aggregation is different from the values being aggregated (e.g. you group together objects), and it should allow you to modify the leftmost argument of each function (mergeCombiners, mergeValue, etc) and return that instead of allocatin

Re: How to specify executor memory in EC2 ?

2014-06-12 Thread Matei Zaharia
t if I find it :) > Thank you, anyway > > > On Wed, Jun 11, 2014 at 12:19 AM, Matei Zaharia > wrote: > It might be that conf/spark-env.sh on EC2 is configured to set it to 512, and > is overriding the application’s settings. Take a look in there and delete > that line

Fwd: ApacheCon CFP closes June 25

2014-06-12 Thread Matei Zaharia
(I’m forwarding this message on behalf of the ApacheCon organizers, who’d like to see involvement from every Apache project!) As you may be aware, ApacheCon will be held this year in Budapest, on November 17-23. (See http://apachecon.eu for more info.) The Call For Papers for that conference is

Re: guidance on simple unit testing with Spark

2014-06-13 Thread Matei Zaharia
You need to factor your program so that it’s not just a main(). This is not a Spark-specific issue, it’s about how you’d unit test any program in general. In this case, your main() creates a SparkContext, so you can’t pass one from outside, and your code has to read data from a file and write it

Re: Is shuffle "stable"?

2014-06-14 Thread Matei Zaharia
The order is not guaranteed actually, only which keys end up in each partition. Reducers may fetch data from map tasks in an arbitrary order, depending on which ones are available first. If you’d like a specific order, you should sort each partition. Here you might be getting it because each par

Re: pyspark serializer can't handle functions?

2014-06-16 Thread Matei Zaharia
It’s true that it can’t. You can try to use the CloudPickle library instead, which is what we use within PySpark to serialize functions (see python/pyspark/cloudpickle.py). However I’m also curious, why do you need an RDD of functions? Matei On Jun 15, 2014, at 4:49 PM, madeleine wrote: > It

Re: pyspark serializer can't handle functions?

2014-06-16 Thread Matei Zaharia
ument functions. > > One other wrinkle is that I'm using alternating minimization, so I'll be > minimizing over the rows and columns of this matrix at alternating steps; > hence I need to store both the matrix and its transpose to avoid data > thrashing. > > > On

Re: Un-serializable 3rd-party classes (Spark, Java)

2014-06-17 Thread Matei Zaharia
There are a few options: - Kryo might be able to serialize these objects out of the box, depending what’s inside them. Try turning it on as described at http://spark.apache.org/docs/latest/tuning.html. - If that doesn’t work, you can create your own “wrapper” objects that implement Serializabl

Re: Spark is now available via Homebrew

2014-06-18 Thread Matei Zaharia
Interesting, does anyone know the people over there who set it up? It would be good if Apache itself could publish packages there, though I’m not sure what’s involved. Since Spark just depends on Java and Python it should be easy for us to update. Matei On Jun 18, 2014, at 1:37 PM, Nick Chamma

Re: Patterns for making multiple aggregations in one pass

2014-06-18 Thread Matei Zaharia
I was going to suggest the same thing :). On Jun 18, 2014, at 4:56 PM, Evan R. Sparks wrote: > This looks like a job for SparkSQL! > > > val sqlContext = new org.apache.spark.sql.SQLContext(sc) > import sqlContext._ > case class MyRecord(country: String, name: String, age: Int, hits: Long) > v

Re: MLLib inside Storm : silly or not ?

2014-06-19 Thread Matei Zaharia
You should be able to use many of the MLlib Model objects directly in Storm, if you save them out using Java serialization. The only one that won’t work is probably ALS, because it’s a distributed model. Otherwise, you will have to output them in your own format and write code for evaluating th

Re: Using Spark

2014-06-21 Thread Matei Zaharia
Alright, added you. On Jun 20, 2014, at 2:52 PM, Ricky Thomas wrote: > Hi, > > Would like to add ourselves to the user list if possible please? > > Company: truedash > url: truedash.io > > Automatic pulling of all your data in to Spark for enterprise visualisation, > predictive analytics an

Re: Powered by Spark addition

2014-06-21 Thread Matei Zaharia
e helping > customer targetting, accurate inventory and efficient analysis. > > Thanks! > > Best Regards, > Sonal > Nube Technologies > > > > > > > On Thu, Jun 12, 2014 at 11:33 PM, Derek Mansen wrote: > Awesome, thank you! > > > On

Re: Shark vs Impala

2014-06-22 Thread Matei Zaharia
In this benchmark, the problem wasn’t that Shark could not run without enough memory; Shark spills some of the data to disk and can run just fine. The issue was that the in-memory form of the RDDs was larger than the cluster’s memory, although the raw Parquet / ORC files did fit in memory, so Cl

Re: Spark Summit 2014 Day 2 Video Streams?

2014-07-01 Thread Matei Zaharia
Yup, we’re going to try to get the videos up as soon as possible. Matei On Jul 1, 2014, at 7:47 PM, Marco Shaw wrote: > They are recorded... For example, 2013: http://spark-summit.org/2013 > > I'm assuming the 2014 videos will be up in 1-2 weeks. > > Marco > > > On Tue, Jul 1, 2014 at 3:18

Re: [ANNOUNCE] Flambo - A Clojure DSL for Apache Spark

2014-07-01 Thread Matei Zaharia
Very cool, Soren, thanks for announcing this! It looks like it didn’t actually require a huge amount of new code either, which is great. Matei On Jul 1, 2014, at 12:31 PM, Soren Macbeth wrote: > Yieldbot is pleased to announce the release of Flambo, our Clojure DSL for > Apache Spark. > > Fl

Re: Spark 1.0: Unable to Read LZO Compressed File

2014-07-01 Thread Matei Zaharia
I’d suggest asking the IBM Hadoop folks, but my guess is that the library cannot be found in /opt/IHC/lib/native/Linux-amd64-64/. Or maybe if this exception is happening in your driver program, the driver program’s java.library.path doesn’t include this. (SPARK_LIBRARY_PATH from spark-env.sh on

Re: java options for spark-1.0.0

2014-07-02 Thread Matei Zaharia
Try looking at the running processes with “ps” to see their full command line and see whether any options are different. It seems like in both cases, your young generation is quite large (11 GB), which doesn’t make lot of sense with a heap of 15 GB. But maybe I’m misreading something. Matei On

Re: Shark Vs Spark SQL

2014-07-02 Thread Matei Zaharia
Spark SQL in Spark 1.1 will include all the functionality in Shark; take a look at http://databricks.com/blog/2014/07/01/shark-spark-sql-hive-on-spark-and-the-future-of-sql-on-spark.html. We decided to do this because at the end of the day, the only code left in Shark was the JDBC / Thrift serv

Re: AWS Credentials for private S3 reads

2014-07-02 Thread Matei Zaharia
When you use hadoopConfiguration directly, I don’t think you have to replace the “/“ with “%2f”. Have you tried it without that? Also make sure you’re not replacing slashes in the URL itself. Matei On Jul 2, 2014, at 4:17 PM, Brian Gawalt wrote: > Hello everyone, > > I'm having some difficul

Re: AWS Credentials for private S3 reads

2014-07-02 Thread Matei Zaharia
Hmm, yeah, that is weird but because it’s only on some files it might mean those didn’t get fully uploaded. Matei On Jul 2, 2014, at 4:50 PM, Brian Gawalt wrote: > HUH; not-scrubbing the slashes fixed it. I would have sworn I tried it, got a > 403 Forbidden, then remembered the slash prescrip

Re: the Pre-built packages for CDH4 can not support yarn ?

2014-07-07 Thread Matei Zaharia
They are for CDH4 without YARN, since YARN is experimental in that. You can download one of the Hadoop 2 packages if you want to run on YARN. Or you might have to build specifically against CDH4's version of YARN if that doesn't work. Matei On Jul 7, 2014, at 9:37 PM, ch huang wrote: > hi,mai

Re: Document page load fault

2014-07-08 Thread Matei Zaharia
Thanks for catching this. For now you can just access the page through http:// instead of https:// to avoid this. Matei On Jul 8, 2014, at 10:46 PM, binbinbin915 wrote: > https://spark.apache.org/docs/latest/mllib-linear-methods.html#logistic-regression > on Chrome 35 with MacX > > Says: > [b

Re: spark ui on yarn

2014-07-12 Thread Matei Zaharia
The UI code is the same in both, but one possibility is that your executors were given less memory on YARN. Can you check that? Or otherwise, how do you know that some RDDs were cached? Matei On Jul 12, 2014, at 4:12 PM, Koert Kuipers wrote: > hey shuo, > so far all stage links work fine for

Re: Can we get a spark context inside a mapper

2014-07-14 Thread Matei Zaharia
You currently can't use SparkContext inside a Spark task, so in this case you'd have to call some kind of local K-means library. One example you can try to use is Weka (http://www.cs.waikato.ac.nz/ml/weka/). You can then load your text files as an RDD of strings with SparkContext.wholeTextFiles

Re: Ideal core count within a single JVM

2014-07-14 Thread Matei Zaharia
Probably something like 8 is best on this kind of machine. What operations are you doing though? It's possible that something else is a contention point at 48 threads, e.g. a common one we've seen is the Linux file system. Matei On Jul 13, 2014, at 4:03 PM, lokesh.gidra wrote: > Hello, > > W

Re: Ideal core count within a single JVM

2014-07-14 Thread Matei Zaharia
Are you increasing the number of parallel tasks with cores as well? With more tasks there will be more data communicated and hence more calls to these functions. Unfortunately contention is kind of hard to measure, since often the result is that you see many cores idle as they're waiting on a l

Re: Memory & compute-intensive tasks

2014-07-14 Thread Matei Zaharia
I think coalesce with shuffle=true will force it to have one task per node. Without that, it might be that due to data locality it decides to launch multiple ones on the same node even though the total # of tasks is equal to the # of nodes. If this is the *only* thing you run on the cluster, yo

  1   2   3   4   >