Re: FW: Re: Autoscaling Spark cluster based on topic sizes/rate of growth in Kafka or Spark's metrics?

2015-05-28 Thread Andrew Or
Hi all, As the author of the dynamic allocation feature I can offer a few insights here. Gerard's explanation was both correct and concise: dynamic allocation is not intended to be used in Spark streaming at the moment (1.4 or before). This is because of two things: (1) Number of receivers is ne

Re: java.io.IOException: FAILED_TO_UNCOMPRESS(5)

2015-06-01 Thread Andrew Or
Hi Deepak, This is a notorious bug that is being tracked at https://issues.apache.org/jira/browse/SPARK-4105. We have fixed one source of this bug (it turns out Snappy had a bug in buffer reuse that caused data corruption). There are other known sources that are being addressed in outstanding patc

Re: [Spark 1.4.0]How to set driver's system property using spark-submit options?

2015-06-12 Thread Andrew Or
Hi Peng, Setting properties through --conf should still work in Spark 1.4. From the warning it looks like the config you are trying to set does not start with the prefix "spark.". What is the config that you are trying to set? -Andrew 2015-06-12 11:17 GMT-07:00 Peng Cheng : > In Spark <1.3.x, t

Re: Dynamic allocator requests -1 executors

2015-06-13 Thread Andrew Or
Hi Patrick, The fix you need is SPARK-6954: https://github.com/apache/spark/pull/5704. If possible, you may cherry-pick the following commit into your Spark deployment and it should resolve the issue: https://github.com/apache/spark/commit/98ac39d2f5828fbdad8c9a4e563ad1169e3b9948 Note that this

Re: PySpark on YARN "port out of range"

2015-06-19 Thread Andrew Or
Hm, one thing to see is whether the same port appears many times (1315905645). The way pyspark works today is that the JVM reads the port from the stdout of the python process. If there is some interference in output from the python side (e.g. any print statements, exception messages), then the Jav

Re: Spark on Yarn - How to configure

2015-06-19 Thread Andrew Or
Hi Ashish, For Spark on YARN, you actually only need the Spark files on one machine - the submission client. This machine could even live outside of the cluster. Then all you need to do is point YARN_CONF_DIR to the directory containing your hadoop configuration files (e.g. yarn-site.xml) on that

Re: Abount Jobs UI in yarn-client mode

2015-06-19 Thread Andrew Or
Did you make sure that the YARN IP is not an internal address? If it still doesn't work then it seems like an issue on the YARN side... 2015-06-19 8:48 GMT-07:00 Sea <261810...@qq.com>: > Hi, all: > I run spark on yarn, I want to see the Jobs UI http://ip:4040/, > but it redirect to http:// > ${

Re: What files/folders/jars spark-submit script depend on ?

2015-06-19 Thread Andrew Or
Hi Elkhan, Spark submit depends on several things: the launcher jar (1.3.0+ only), the spark-core jar, and the spark-yarn jar (in your case). Why do you want to put it in HDFS though? AFAIK you can't execute scripts directly from HDFS; you need to copy them to a local file system first. I don't se

Re: Submitting Spark Applications using Spark Submit

2015-06-19 Thread Andrew Or
Hi Raghav, If you want to make changes to Spark and run your application with it, you may follow these steps. 1. git clone g...@github.com:apache/spark 2. cd spark; build/mvn clean package -DskipTests [...] 3. make local changes 4. build/mvn package -DskipTests [...] (no need to clean again here)

Re: Submitting Spark Applications using Spark Submit

2015-06-19 Thread Andrew Or
Thanks, > Raghav > > > On Friday, June 19, 2015, Andrew Or wrote: > >> Hi Raghav, >> >> If you want to make changes to Spark and run your application with it, >> you may follow these steps. >> >> 1. git clone g...@github.com:apache/spark >

Re: PySpark on YARN "port out of range"

2015-06-22 Thread Andrew Or
is actually outputting (in hopes > that yields a clue)? > > On Jun 19, 2015, at 6:47 PM, Andrew Or wrote: > > Hm, one thing to see is whether the same port appears many times (1315905645). > The way pyspark works today is that the JVM reads the port from the stdout > of the

Re: Submitting Spark Applications using Spark Submit

2015-06-22 Thread Andrew Or
amazonaws.com/10.165.103.16:7077 > <http://ec2-XXX.compute-1.amazonaws.com/10.165.103.16:7077> *I don’t > understand where it gets the *10.165.103.16 > <http://ec2-XXX.compute-1.amazonaws.com/10.165.103.16:7077> *from. I > never specify that in the master url command line par

Re: Disable heartbeat messages in REPL

2015-07-08 Thread Andrew Or
Hi Lincoln, I've noticed this myself. I believe it's a new issue that only affects local mode. I've filed a JIRA to track it: https://issues.apache.org/jira/browse/SPARK-8911 2015-07-08 14:20 GMT-07:00 Lincoln Atkinson : > Brilliant! Thanks. > > > > *From:* Feynman Liang [mailto:fli...@databrick

Re: Spark serialization in closure

2015-07-09 Thread Andrew Or
Hi Chen, I believe the issue is that `object foo` is a member of `object testing`, so the only way to access `object foo` is to first pull `object testing` into the closure, then access a pointer to get to `object foo`. There are two workarounds that I'm aware of: (1) Move `object foo` outside of

Re: spark-submit

2015-07-10 Thread Andrew Or
Hi Ashutosh, I believe the class is org.apache.spark.*examples.*graphx.Analytics? If you're running page rank on live journal you could just use org.apache.spark.examples.graphx.LiveJournalPageRank. -Andrew 2015-07-10 3:42 GMT-07:00 AshutoshRaghuvanshi < ashutosh.raghuvans...@gmail.com>: > when

Re: Starting Spark-Application without explicit submission to cluster?

2015-07-10 Thread Andrew Or
Hi Jan, Most SparkContext constructors are there for legacy reasons. The point of going through spark-submit is to set up all the classpaths, system properties, and resolve URIs properly *with respect to the deployment mode*. For instance, jars are distributed differently between YARN cluster mode

Re: How to restrict disk space for spark caches on yarn?

2015-07-10 Thread Andrew Or
Hi Peter, AFAIK Spark assumes infinite disk space, so there isn't really a way to limit how much space it uses. Unfortunately I'm not aware of a simpler workaround than to simply provision your cluster with more disk space. By the way, are you sure that it's disk space that exceeded the limit, but

Re: Unable to use dynamicAllocation if spark.executor.instances is set in spark-defaults.conf

2015-07-15 Thread Andrew Or
Yeah, we could make it a log a warning instead. 2015-07-15 14:29 GMT-07:00 Kelly, Jonathan : > Thanks! Is there an existing JIRA I should watch? > > > ~ Jonathan > > From: Sandy Ryza > Date: Wednesday, July 15, 2015 at 2:27 PM > To: Jonathan Kelly > Cc: "user@spark.apache.org" > Subject: R

Re: The auxService:spark_shuffle does not exist

2015-07-17 Thread Andrew Or
Hi all, Did you forget to restart the node managers after editing yarn-site.xml by any chance? -Andrew 2015-07-17 8:32 GMT-07:00 Andrew Lee : > I have encountered the same problem after following the document. > > Here's my spark-defaults.conf > > spark.shuffle.service.enabled true > spark.dyna

Re: Using dynamic allocation and shuffle service in Standalone Mode

2016-03-08 Thread Andrew Or
Hi Yuval, if you start the Workers with `spark.shuffle.service.enabled = true` then the workers will each start a shuffle service automatically. No need to start the shuffle services yourself separately. -Andrew 2016-03-08 11:21 GMT-08:00 Silvio Fiorito : > There’s a script to start it up under

Re: No event log in /tmp/spark-events

2016-03-08 Thread Andrew Or
Hi Patrick, I think he means just write `/tmp/sparkserverlog` instead of `file:/tmp/sparkserverlog`. However, I think both should work. What mode are you running in, client mode (the default) or cluster mode? If the latter your driver will be run on the cluster, and so your event logs won't be on

Re: Limit of application submission to cluster

2015-12-18 Thread Andrew Or
Hi Saif, have you verified that the cluster has enough resources for all 4 programs? -Andrew 2015-12-18 5:52 GMT-08:00 : > Hello everyone, > > I am testing some parallel program submission to a stand alone cluster. > Everything works alright, the problem is, for some reason, I can’t submit > mor

Re: Yarn application ID for Spark job on Yarn

2015-12-18 Thread Andrew Or
Hi Roy, I believe Spark just gets its application ID from YARN, so you can just do `sc.applicationId`. -Andrew 2015-12-18 0:14 GMT-08:00 Deepak Sharma : > I have never tried this but there is yarn client api's that you can use in > your spark program to get the application id. > Here is the lin

Re: imposed dynamic resource allocation

2015-12-18 Thread Andrew Or
Hi Antony, The configuration to enable dynamic allocation is per-application. If you only wish to enable this for one of your applications, just set `spark.dynamicAllocation.enabled` to true for that application only. The way it works under the hood is that application will start sending requests

Re: which aws instance type for shuffle performance

2015-12-18 Thread Andrew Or
Hi Rastan, Unless you're using off-heap memory or starting multiple executors per machine, I would recommend the r3.2xlarge option, since you don't actually want gigantic heaps (100GB is more than enough). I've personally run Spark on a very large scale with r3.8xlarge instances, but I've been usi

Re: Can't submit job to stand alone cluster

2015-12-29 Thread Andrew Or
Hi Greg, It's actually intentional for standalone cluster mode to not upload jars. One of the reasons why YARN takes at least 10 seconds before running any simple application is because there's a lot of random overhead (e.g. putting jars in HDFS). If this missing functionality is not documented so

Re: Can't submit job to stand alone cluster

2015-12-29 Thread Andrew Or
thread in ApplicationMaster; use --jars > option with a globally visible path to said jar > 3. Yarn Client-mode: client and driver run on the same machine. driver > is *NOT* a thread in ApplicationMaster; use --packages to submit a jar > > > On Tuesday, December 29, 2015 1:54 PM,

Re: Opening Dynamic Scaling Executors on Yarn

2015-12-29 Thread Andrew Or
> > External shuffle service is backward compatible, so if you deployed 1.6 > shuffle service on NM, it could serve both 1.5 and 1.6 Spark applications. Actually, it just happens to be backward compatible because we didn't change the shuffle file formats. This may not necessarily be the case movi

Re: Can't submit job to stand alone cluster

2015-12-29 Thread Andrew Or
http://www.cloudera.com/content/www/en-us/documentation/enterprise/latest/topics/cdh_ig_running_spark_apps.html> > Preview by Yahoo > > > > > On Tuesday, December 29, 2015 2:42 PM, Andrew Or > wrote: > > > The confusion here is the expression "standalone cluster mo

Re: Can't submit job to stand alone cluster

2015-12-29 Thread Andrew Or
application depends on, you should specify them through the > --jars flag using comma as a delimiter (e.g. --jars jar1,jar2). > > That can't be true; this is only the case when Spark runs on top of YARN. > Please correct me, if I'm wrong. > > Thanks > &

Re: Can't submit job to stand alone cluster

2015-12-30 Thread Andrew Or
ved > > Sent from my iPhone > > On Dec 29, 2015, at 2:43 PM, Annabel Melongo < > melongo_anna...@yahoo.com> wrote: > > Thanks Andrew for this awesome explanation [image: *:) happy] > > > On Tuesday, December 29, 2015 5:30 PM, Andrew Or < > and...@databricks.c

Re: Read Accumulator value while running

2016-01-13 Thread Andrew Or
Hi Kira, As you suspected, accumulator values are only updated after the task completes. We do send accumulator updates from the executors to the driver on periodic heartbeats, but these only concern internal accumulators, not the ones created by the user. In short, I'm afraid there is not curren

Re: automatically unpersist RDDs which are not used for 24 hours?

2016-01-13 Thread Andrew Or
Hi Alex, Yes, you can set `spark.cleaner.ttl`: http://spark.apache.org/docs/1.6.0/configuration.html, but I would not recommend it! We are actually removing this property in Spark 2.0 because it has caused problems for many users in the past. In particular, if you accidentally use a variable that

Re: [Spark on YARN] Multiple Auxiliary Shuffle Service Versions

2015-10-05 Thread Andrew Or
Hi all, Both the history server and the shuffle service are backward compatible, but not forward compatible. This means as long as you have the latest version of history server / shuffle service running in your cluster then you're fine (you don't need multiple of them). That said, an old shuffle

Re: Spark 1.5.1 Dynamic Resource Allocation

2015-11-09 Thread Andrew Or
Hi Tom, I believe a workaround is to set `spark.dynamicAllocation.initialExecutors` to 0. As others have mentioned, from Spark 1.5.2 onwards this should no longer be necessary. -Andrew 2015-11-09 8:19 GMT-08:00 Jonathan Kelly : > Tom, > > You might be hitting https://issues.apache.org/jira/brow

Re: create a table for csv files

2015-11-19 Thread Andrew Or
There's not an easy way. The closest thing you can do is: import org.apache.spark.sql.functions._ val df = ... df.withColumn("id", monotonicallyIncreasingId()) -Andrew 2015-11-19 8:23 GMT-08:00 xiaohe lan : > Hi, > > I have some csv file in HDFS with headers like col1, col2, col3, I want to >

Re: Warning: Master endpoint spark://ip:7077 was not a REST server. Falling back to legacy submission gateway instead.

2015-12-10 Thread Andrew Or
Hi Andy, You must be running in cluster mode. The Spark Master accepts client mode submissions on port 7077 and cluster mode submissions on port 6066. This is because standalone cluster mode uses a REST API to submit applications by default. If you submit to port 6066 instead the warning should go

Re: Spark job submission REST API

2015-12-10 Thread Andrew Or
Hello, The hidden API was implemented for use internally and there are no plans to make it public at this point. It was originally introduced to provide backward compatibility in submission protocol across multiple versions of Spark. A full-fledged stable REST API for submitting applications would

Re: The auxService:spark_shuffle does not exist

2015-07-21 Thread Andrew Or
Hi Andrew, Based on your driver logs, it seems the issue is that the shuffle service is actually not running on the NodeManagers, but your application is trying to provide a "spark_shuffle" secret anyway. One way to verify whether the shuffle service is actually started is to look at the NodeManag

Re: Spark spark.shuffle.memoryFraction has no affect

2015-07-22 Thread Andrew Or
Hi, The setting of 0.2 / 0.6 looks reasonable to me. Since you are not using caching at all, have you tried trying something more extreme, like 0.1 / 0.9? Since disabling spark.shuffle.spill didn't cause an OOM this setting should be fine. Also, one thing you could do is to verify the shuffle byte

Re: How to share a Map among RDDS?

2015-07-22 Thread Andrew Or
Hi Dan, If the map is small enough, you can just broadcast it, can't you? It doesn't have to be an RDD. Here's an example of broadcasting an array and using it on the executors: https://github.com/apache/spark/blob/c03299a18b4e076cabb4b7833a1e7632c5c0dabe/examples/src/main/scala/org/apache/spark/e

Re: Which memory fraction is Spark using to compute RDDs that are not going to be persisted

2015-07-22 Thread Andrew Or
Hi, It would be whatever's left in the JVM. This is not explicitly controlled by a fraction like storage or shuffle. However, the computation usually doesn't need to use that much space. In my experience it's almost always the caching or the aggregation during shuffles that's the most memory inten

Re: spark.deploy.spreadOut core allocation

2015-07-22 Thread Andrew Or
Hi Srikanth, It does look like a bug. Did you set `spark.executor.cores` in your application by any chance? -Andrew 2015-07-22 8:05 GMT-07:00 Srikanth : > Hello, > > I've set spark.deploy.spreadOut=false in spark-env.sh. > >> export SPARK_MASTER_OPTS="-Dspark.deploy.defaultCores=4 >> -Dspark.de

Re: spark.deploy.spreadOut core allocation

2015-07-22 Thread Andrew Or
22 11:49 GMT-07:00 Andrew Or : > Hi Srikanth, > > It does look like a bug. Did you set `spark.executor.cores` in your > application by any chance? > > -Andrew > > 2015-07-22 8:05 GMT-07:00 Srikanth : > >> Hello, >> >> I've set spark.deploy.spreadOut

Re: How to share a Map among RDDS?

2015-07-22 Thread Andrew Or
Cheers, > Dan > > > > 2015-07-22 2:20 GMT-05:00 Andrew Or : > >> Hi Dan, >> >> If the map is small enough, you can just broadcast it, can't you? It >> doesn't have to be an RDD. Here's an example of broadcasting an array and >> using it o

Re: spark.executor.memory and spark.driver.memory have no effect in yarn-cluster mode (1.4.x)?

2015-07-22 Thread Andrew Or
Hi Michael, In general, driver related properties should not be set through the SparkConf. This is because by the time the SparkConf is created, we have already started the driver JVM, so it's too late to change the memory, class paths and other properties. In cluster mode, executor related prope

Re: No event logs in yarn-cluster mode

2015-08-01 Thread Andrew Or
Hi Akmal, It might be on HDFS, since you provided a relative path /opt/spark/spark-events to `spark.eventLog.dir`. -Andrew 2015-08-01 9:25 GMT-07:00 Akmal Abbasov : > Hi, I am trying to configure a history server for application. > When I running locally(./run-example SparkPi), the event logs a

Re: Spark master driver UI: How to keep it after process finished?

2015-08-08 Thread Andrew Or
Hi Saif, You need to run your application with `spark.eventLog.enabled` set to true. Then if you are using standalone mode, you can view the Master UI at port 8080. Otherwise, you may start a history server through `sbin/start-history-server.sh`, which by default starts the history UI at port 1808

Re: TestSQLContext compilation error when run SparkPi in Intellij ?

2015-08-15 Thread Andrew Or
Hi Canan, TestSQLContext is no longer a singleton but now a class. It is never meant to be a fully public API, but if you wish to use it you can just instantiate a new one: val sqlContext = new TestSQLContext or just create a new SQLContext from a SparkContext. -Andrew 2015-08-15 20:33 GMT-07:0

Re: Why standalone mode don't allow to set num-executor ?

2015-08-18 Thread Andrew Or
Hi Canan, This is mainly for legacy reasons. The default behavior in standalone in mode is that the application grabs all available resources in the cluster. This effectively means we want one executor per worker, where each executor grabs all the available cores and memory on that worker. In this

Re: how do I execute a job on a single worker node in standalone mode

2015-08-18 Thread Andrew Or
Hi Axel, You can try setting `spark.deploy.spreadOut` to false (through your conf/spark-defaults.conf file). What this does is essentially try to schedule as many cores on one worker as possible before spilling over to other workers. Note that you *must* restart the cluster through the sbin script

Re: Programmatically create SparkContext on YARN

2015-08-18 Thread Andrew Or
Hi Andreas, I believe the distinction is not between standalone and YARN mode, but between client and cluster mode. In client mode, your Spark submit JVM runs your driver code. In cluster mode, one of the workers (or NodeManagers if you're using YARN) in the cluster runs your driver code. In the

Re: Difference between Sort based and Hash based shuffle

2015-08-18 Thread Andrew Or
Hi Muhammad, On a high level, in hash-based shuffle each mapper M writes R shuffle files, one for each reducer where R is the number of reduce partitions. This results in M * R shuffle files. Since it is not uncommon for M and R to be O(1000), this quickly becomes expensive. An optimization with h

Re: dse spark-submit multiple jars issue

2015-08-18 Thread Andrew Or
Hi Satish, The problem is that `--jars` accepts a comma-delimited list of jars! E.g. spark-submit ... --jars lib1.jar,lib2.jar,lib3.jar main.jar where main.jar is your main application jar (the one that starts a SparkContext), and lib*.jar refer to additional libraries that your main application

Re: Difference between Sort based and Hash based shuffle

2015-08-19 Thread Andrew Or
gt; So the reason why key value pairs with same keys are always found in a > single buckets in Hash based shuffle but not in Sort is because in > sort-shuffle each mapper writes a single partitioned file, and it is up to > the reducer to fetch correct partitions from the the files ? > > On

Re: how do I execute a job on a single worker node in standalone mode

2015-08-19 Thread Andrew Or
rything on 1 node, it looks like it's not grabbing the extra nodes. > > On Wed, Aug 19, 2015 at 8:43 AM, Axel Dahl wrote: > >> That worked great, thanks Andrew. >> >> On Tue, Aug 18, 2015 at 1:39 PM, Andrew Or wrote: >> >>> Hi Axel, >>> >

Re: Why use spark.history.fs.logDirectory instead of spark.eventLog.dir

2015-08-19 Thread Andrew Or
Hi Canan, The event log dir is a per-application setting whereas the history server is an independent service that serves history UIs from many applications. If you use history server locally then the `spark.history.fs.logDirectory` will happen to point to `spark.eventLog.dir`, but the use case it

Re: DAG related query

2015-08-20 Thread Andrew Or
Hi Bahubali, Once RDDs are created, they are immutable (in most cases). In your case you end up with 3 RDDs: (1) the original rdd1 that reads from the text file (2) rdd2, that applies a map function on (1), and (3) the new rdd1 that applies a map function on (2) There's no cycle because you have

Re: Spark ec2 lunch problem

2015-08-24 Thread Andrew Or
Hey Garry, Have you verified that your particular VPC and subnet are open to the world? In particular, have you verified the route table attached to your VPC / subnet contains an internet gateway open to the public? I've run into this issue myself recently and that was the problem for me. -Andre

Re: Why are executors on slave never used?

2015-09-21 Thread Andrew Or
Hi Joshua, What cluster manager are you using, standalone or YARN? (Note that standalone here does not mean local mode). If standalone, you need to do `setMaster("spark://[CLUSTER_URL]:7077")`, where CLUSTER_URL is the machine that started the standalone Master. If YARN, you need to do `setMaster

Re: PySpark Client

2015-01-20 Thread Andrew Or
Hi Chris, Short answer is no, not yet. Longer answer is that PySpark only supports client mode, which means your driver runs on the same machine as your submission client. By corollary this means your submission client must currently depend on all of Spark and its dependencies. There is a patch t

Re: spark-submit --py-files remote: "Only local additional python files are supported"

2015-01-20 Thread Andrew Or
Hi Vladimir, Yes, as the error messages suggests, PySpark currently only supports local files. This does not mean it only runs in local mode, however; you can still run PySpark on any cluster manager (though only in client mode). All this means is that your python files must be on your local file

Re: Aggregate order semantics when spilling

2015-01-20 Thread Andrew Or
Hi Justin, I believe the intended semantics of groupByKey or cogroup is that the ordering *within a key *is not preserved if you spill. In fact, the test cases for the ExternalAppendOnlyMap only assert that the Set representation of the results is as expected (see this line

Re: Which version to use for shuffle service if I'm going to run multiple versions of Spark

2015-02-11 Thread Andrew Or
Hi Jianshi, For YARN, there may be an issue with how a recently patch changes the accessibility of the shuffle files by the external shuffle service: https://issues.apache.org/jira/browse/SPARK-5655. It is likely that you will hit this with 1.2.1, actually. For this reason I would have to recommen

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

2014-06-02 Thread Andrew Or
; >> $ PYTHONPATH=/path/to/assembly/jar python > >> >> >>> import pyspark > >> >> > >> >> This following pull request did mention something about generating a > >> >> zip > >> >> file for all python related modules: >

Re: Spark Logging

2014-06-10 Thread Andrew Or
You can import org.apache.spark.Logging, and use logInfo, logWarning etc. Besides viewing them from the Web console, the location of the logs can be found under $SPARK_HOME/logs, on both the driver and executor machines. (If you are on YARN, these logs are located elsewhere, however.) 2014-06-10

Re: Can't find pyspark when using PySpark on YARN

2014-06-10 Thread Andrew Or
Hi Qi Ping, You don't have to distribute these files; they are automatically packaged in the assembly jar, which is already shipped to the worker nodes. Other people have run into the same issue. See if the instructions here are of any help: http://mail-archives.apache.org/mod_mbox/spark-user/201

Re: problem starting the history server on EC2

2014-06-10 Thread Andrew Or
Can you try file:/root/spark_log? 2014-06-10 19:22 GMT-07:00 zhen : > I checked the permission on root and it is the following: > > drwxr-xr-x 20 root root 4096 Jun 11 01:05 root > > So anyway, I changed to use /tmp/spark_log instead and this time I made > sure > that all permissions are given

Re: problem starting the history server on EC2

2014-06-10 Thread Andrew Or
No, I meant pass the path to the history server start script. 2014-06-10 19:33 GMT-07:00 zhen : > Sure here it is: > > drwxrwxrwx 2 1000 root 4096 Jun 11 01:05 spark_logs > > Zhen > > > > -- > View this message in context: > http://apache-spark-user-list.1001560.n3.nabble.com/problem-starting-t

Re: Adding external jar to spark-shell classpath in spark 1.0

2014-06-11 Thread Andrew Or
This is a known issue: https://issues.apache.org/jira/browse/SPARK-1919. We haven't found a fix yet, but for now, you can workaround this by including your simple class in your application jar. 2014-06-11 10:25 GMT-07:00 Ulanov, Alexander : > Hi, > > > > I am currently using spark 1.0 locally o

Re: Adding external jar to spark-shell classpath in spark 1.0

2014-06-11 Thread Andrew Or
Ulanov, Alexander : > Could you elaborate on this? I don’t have an application, I just use > spark shell. > > > > *From:* Andrew Or [mailto:and...@databricks.com] > *Sent:* Wednesday, June 11, 2014 9:40 PM > > *To:* user@spark.apache.org > *Subject:* Re: Adding external j

Re: Spark 1.0.0 Standalone AppClient cannot connect Master

2014-06-12 Thread Andrew Or
Hi Wang Hao, This is not removed. We moved it here: http://spark.apache.org/docs/latest/hadoop-third-party-distributions.html If you're building with SBT, and you don't specify the SPARK_HADOOP_VERSION, then it defaults to 1.0.4. Andrew 2014-06-12 6:24 GMT-07:00 Hao Wang : > Hi, all > > Why do

Re: use spark-shell in the source

2014-06-12 Thread Andrew Or
Not sure if this is what you're looking for, but have you looked at java's ProcessBuilder? You can do something like for (line <- lines) { val command = line.split(" ") // You may need to deal with quoted strings val process = new ProcessBuilder(command) // redirect output of process to main

Re: spark master UI does not keep detailed application history

2014-06-16 Thread Andrew Or
Are you referring to accessing a SparkUI for an application that has finished? First you need to enable event logging while the application is still running. In Spark 1.0, you set this by adding a line to $SPARK_HOME/conf/spark-defaults.conf: spark.eventLog.enabled true Other than that, the conte

Re: Yarn-client mode and standalone-client mode hang during job start

2014-06-17 Thread Andrew Or
Standalone-client mode is not officially supported at the moment. For standalone-cluster and yarn-client modes, however, they should work. For both modes, are you running spark-submit from within the cluster, or outside of it? If the latter, could you try running it from within the cluster and see

Re: join operation is taking too much time

2014-06-17 Thread Andrew Or
How long does it get stuck for? This is a common sign for the OS thrashing due to out of memory exceptions. If you keep it running longer, does it throw an error? Depending on how large your other RDD is (and your join operation), memory pressure may or may not be the problem at all. It could be t

Re: little confused about SPARK_JAVA_OPTS alternatives

2014-06-19 Thread Andrew Or
Hi Koert and Lukasz, The recommended way of not hard-coding configurations in your application is through conf/spark-defaults.conf as documented here: http://spark.apache.org/docs/latest/configuration.html#dynamically-loading-spark-properties. However, this is only applicable to spark-submit, so t

Re: Getting started : Spark on YARN issue

2014-06-19 Thread Andrew Or
Hi Praveen, Yes, the fact that it is trying to use a private IP from outside of the cluster is suspicious. My guess is that your HDFS is configured to use internal IPs rather than external IPs. This means even though the hadoop confs on your local machine only use external IPs, the org.apache.spar

Re: Getting started : Spark on YARN issue

2014-06-19 Thread Andrew Or
(Also, an easier workaround is to simply submit the application from within your cluster, thus saving you all the manual labor of reconfiguring everything to use public hostnames. This may or may not be applicable to your use case.) 2014-06-19 14:04 GMT-07:00 Andrew Or : > Hi Praveen, >

Re: options set in spark-env.sh is not reflecting on actual execution

2014-06-20 Thread Andrew Or
Hi Meethu, Are you using Spark 1.0? If so, you should use spark-submit ( http://spark.apache.org/docs/latest/submitting-applications.html), which has --executor-memory. If you don't want to specify this every time you submit an application, you can also specify spark.executor.memory in $SPARK_HOME

Re: little confused about SPARK_JAVA_OPTS alternatives

2014-06-20 Thread Andrew Or
gt; to launch it. thats not very scalable, since i can only launch it one way >> in the end... >> >> >> On Thu, Jun 19, 2014 at 4:58 PM, Andrew Or wrote: >> >>> Hi Koert and Lukasz, >>> >>> The recommended way of not hard-coding configurations i

Re: hi

2014-06-23 Thread Andrew Or
Hm, spark://localhost:7077 should work, because the standalone master binds to 0.0.0.0. Are you sure you ran `sbin/start-master.sh`? 2014-06-22 22:50 GMT-07:00 Akhil Das : > Open your webUI in the browser and see the spark url in the top left > corner of the page and use it while starting your s

Re: Error in run spark.ContextCleaner under Spark 1.0.0

2014-06-23 Thread Andrew Or
Hi Haoming, You can safely disregard this error. This is printed at the end of the execution when we clean up and kill the daemon context cleaning thread. In the future it would be good to silence this particular message, as it may be confusing to users. Andrew 2014-06-23 12:13 GMT-07:00 Haomin

Re: hi

2014-06-23 Thread Andrew Or
Ah never mind. The 0.0.0.0 is for the UI, not for Master, which uses the output of the "hostname" command. But yes, long answer short, go to the web UI and use that URL. 2014-06-23 11:13 GMT-07:00 Andrew Or : > Hm, spark://localhost:7077 should work, because the standalone mast

Re: problem about cluster mode of spark 1.0.0

2014-06-24 Thread Andrew Or
Hi Randy and Gino, The issue is that standalone-cluster mode is not officially supported. Please use standalone-client mode instead, i.e. specify --deploy-mode client in spark-submit, or simply leave out this config because it defaults to client mode. Unfortunately, this is not currently document

Re: spark streaming, kafka, SPARK_CLASSPATH

2014-06-24 Thread Andrew Or
Hi all, The short answer is that standalone-cluster mode through spark-submit is broken (and in fact not officially supported). Please use standalone-client mode instead. The long answer is provided here: http://mail-archives.apache.org/mod_mbox/spark-user/201406.mbox/%3cCAMJOb8m6gF9B3W=p12hi88mex

Re: Does PUBLIC_DNS environment parameter really works?

2014-06-24 Thread Andrew Or
Hi Peng, What you're looking for is SPARK_MASTER_IP, which defaults to the output of the command "hostname" (see sbin/start-master.sh). What SPARK_PUBLIC_DNS does is it changes what the Master or the Worker advertise to others. If this is set, the links on the Master and Worker web UI will use pu

Re: Spark 1.0.0 on yarn cluster problem

2014-06-25 Thread Andrew Or
Hi Sophia, did you ever resolve this? A common cause for not giving resources to the job is that the RM cannot communicate with the workers. This itself has many possible causes. Do you have a full stack trace from the logs? Andrew 2014-06-13 0:46 GMT-07:00 Sophia : > With the yarn-client mode

Re: About StorageLevel

2014-06-26 Thread Andrew Or
Hi Kang, You raise a good point. Spark does not automatically cache all your RDDs. Why? Simply because the application may create many RDDs, and not all of them are to be reused. After all, there is only so much memory available to each executor, and caching an RDD adds some overhead especially if

Re: Run spark unit test on Windows 7

2014-07-02 Thread Andrew Or
Hi Konstatin, We use hadoop as a library in a few places in Spark. I wonder why the path includes "null" though. Could you provide the full stack trace? Andrew 2014-07-02 9:38 GMT-07:00 Konstantin Kudryavtsev < kudryavtsev.konstan...@gmail.com>: > Hi all, > > I'm trying to run some transforma

Re: Help alleviating OOM errors

2014-07-02 Thread Andrew Or
Hi Yana, In 0.9.1, spark.shuffle.spill is set to true by default so you shouldn't need to manually set it. Here are a few common causes of OOMs in Spark: - Too few partitions: if one partition is too big, it may cause an OOM if there is not enough space to unroll the entire partition in memory.

Re: NullPointerException on ExternalAppendOnlyMap

2014-07-02 Thread Andrew Or
Hi Konstantin, Thanks for reporting this. This happens because there are null keys in your data. In general, Spark should not throw null pointer exceptions, so this is a bug. I have fixed this here: https://github.com/apache/spark/pull/1288. For now, you can workaround this by special-handling yo

Re: write event logs with YARN

2014-07-03 Thread Andrew Or
Hi Christophe, another Andrew speaking. Your configuration looks fine to me. From the stack trace it seems that we are in fact closing the file system pre-maturely elsewhere in the system, such that when it tries to write the APPLICATION_COMPLETE file it throws the exception you see. This does loo

Re: tiers of caching

2014-07-07 Thread Andrew Or
Others have also asked for this on the mailing list, and hence there's a related JIRA: https://issues.apache.org/jira/browse/SPARK-1762. Ankur brings up a good point in that any current implementation of in-memory shuffles will compete with application RDD blocks. I think we should definitely add t

Re: Issues in opening UI when running Spark Streaming in YARN

2014-07-07 Thread Andrew Or
I will assume that you are running in yarn-cluster mode. Because the driver is launched in one of the containers, it doesn't make sense to expose port 4040 for the node that contains the container. (Imagine if multiple driver containers are launched on the same node. This will cause a port collisio

Re: Issues in opening UI when running Spark Streaming in YARN

2014-07-07 Thread Andrew Or
for my co-worker on Pivotal HD cluster, therefore I am > still looking what's the difference in terms of cluster setup or something > else. > > > Chester > > > > > > On Mon, Jul 7, 2014 at 11:42 AM, Andrew Or wrote: > >> I will assume that you are r

Re: Scheduling in spark

2014-07-08 Thread Andrew Or
Here's the most updated version of the same page: http://spark.apache.org/docs/latest/job-scheduling 2014-07-08 12:44 GMT-07:00 Sujeet Varakhedi : > This is a good start: > > http://www.eecs.berkeley.edu/~tdas/spark_docs/job-scheduling.html > > > On Tue, Jul 8, 2014 at 9:11 AM, rapelly kartheek

Re: Spark: All masters are unresponsive!

2014-07-08 Thread Andrew Or
It seems that your driver (which I'm assuming you launched on the master node) can now connect to the Master, but your executors cannot. Did you make sure that all nodes have the same conf/spark-defaults.conf, conf/spark-env.sh, and conf/slaves? It would be good if you can post the stderr of the ex

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

2014-07-08 Thread Andrew Or
Hi Mikhail, It looks like the documentation is a little out-dated. Neither is true anymore. In general, we try to shift away from short options ("-em", "-dm" etc.) in favor of more explicit ones ("--executor-memory", "--driver-memory"). These options, and "--cores", refer to the arguments passed i

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

2014-07-08 Thread Andrew Or
>> "The proper way to specify this is through "spark.master" in your config or the "--master" parameter to spark-submit." By "this" I mean configuring which master the driver connects to (not which port and address the standalone Master binds to).

  1   2   3   >