Re: Execution stalls in LogisticRegressionWithSGD

2014-07-02 Thread Xiangrui Meng
Could you check the driver memory in the executor tab of the Spark UI when the job is running? If it is too small, please set --driver-memory with spark-submit, e.g. 10g. Could you also attach the master log under spark/logs as well? -Xiangrui On Wed, Jul 2, 2014 at 9:34 AM, Bharath Ravi Kumar wr

Re: SparkKMeans.scala from examples will show: NoClassDefFoundError: breeze/linalg/Vector

2014-07-02 Thread Xiangrui Meng
The SparkKMeans is just an example code showing a barebone implementation of k-means. To run k-means on big datasets, please use the KMeans implemented in MLlib directly: http://spark.apache.org/docs/latest/mllib-clustering.html -Xiangrui On Wed, Jul 2, 2014 at 9:50 AM, Wanda Hawk wrote: > I can

Re: Run spark unit test on Windows 7

2014-07-02 Thread Konstantin Kudryavtsev
It sounds really strange... I guess it is a bug, critical bug and must be fixed... at least some flag must be add (unable.hadoop) I found the next workaround : 1) download compiled winutils.exe from http://social.msdn.microsoft.com/Forums/windowsazure/en-US/28a57efb-082b-424b-8d9e-731b1fe135de/pl

Re: Integrate Spark Editor with Hue for source compiled installation of spark/spark-jobServer

2014-07-02 Thread Sunita Arvind
That's good to know. I will try it out. Thanks Romain On Friday, June 27, 2014, Romain Rigaux wrote: > So far Spark Job Server does not work with Spark 1.0: > https://github.com/ooyala/spark-jobserver > > So this works only with Spark 0.9 currently: > > http://gethue.com/get-started-with-spark-

Re: MLLib : Math on Vector and Matrix

2014-07-02 Thread Xiangrui Meng
Hi Dmitriy, It is sweet to have the bindings, but it is very easy to downgrade the performance with them. The BLAS/LAPACK APIs have been there for more than 20 years and they are still the top choice for high-performance linear algebra. I'm thinking about whether it is possible to make the evaluat

Re: MLLib : Math on Vector and Matrix

2014-07-02 Thread Xiangrui Meng
Hi Thunder, Please understand that both MLlib and breeze are in active development. Before v1.0, we used jblas but in the public APIs we only exposed Array[Double]. In v1.0, we introduced Vector that supports both dense and sparse data and switched the backend to breeze/netlib-java (except ALS). W

Spark S3 LZO input files

2014-07-02 Thread hassan
I'm trying to read input files from S3. The files are compressed using LZO. i-e from spark-shell sc.textFile("s3n://path/xx.lzo").first returns 'String = �LZO?' Spark does not uncompress the data from the file. I am using cloudera manager 5, with CDH 5.0.2. I've already installed 'GPLEXTRAS' par

Re: One question about RDD.zip function when trying Naive Bayes

2014-07-02 Thread x
Thanks for the confirm. I will be checking it. Regards, xj On Thu, Jul 3, 2014 at 2:31 PM, Xiangrui Meng wrote: > This is due to a bug in sampling, which was fixed in 1.0.1 and latest > master. See https://github.com/apache/spark/pull/1234 . -Xiangrui > > On Wed, Jul 2, 2014 at 8:23 PM, x wro

Re: One question about RDD.zip function when trying Naive Bayes

2014-07-02 Thread Xiangrui Meng
This is due to a bug in sampling, which was fixed in 1.0.1 and latest master. See https://github.com/apache/spark/pull/1234 . -Xiangrui On Wed, Jul 2, 2014 at 8:23 PM, x wrote: > Hello, > > I a newbie to Spark MLlib and ran into a curious case when following the > instruction at the page below. >

One question about RDD.zip function when trying Naive Bayes

2014-07-02 Thread x
Hello, I a newbie to Spark MLlib and ran into a curious case when following the instruction at the page below. http://spark.apache.org/docs/latest/mllib-naive-bayes.html I ran a test program on my local machine using some data. val spConfig = (new SparkConf).setMaster("local").setAppName("Spark

Re: Distribute data from Kafka evenly on cluster

2014-07-02 Thread Tobias Pfeiffer
Thank you very much for the link, that was very helpful! So, apparently the `topics: Map[String, Int]` parameter controls the number of partitions that the data is initially added to; the number N in val kafkaInputs = (1 to N).map { _ => ssc.kafkaStream(zkQuorum, groupId, Map("topic" -> 1))

Re: Kafka - streaming from multiple topics

2014-07-02 Thread Tobias Pfeiffer
Sergey, you might actually consider using two streams, like * val stream1 = KafkaUtils.createStream(ssc,"localhost:2181","logs", Map("retarget" -> 2))* * val stream2 = KafkaUtils.createStream(ssc,"localhost:2181","logs", Map("datapair" -> 2))* to achieve what you want. This has the addit

Re: Use Spark Streaming to update result whenever data come

2014-07-02 Thread Tobias Pfeiffer
Bill, can't you just add more nodes in order to speed up the processing? Tobias On Thu, Jul 3, 2014 at 7:09 AM, Bill Jay wrote: > Hi all, > > I have a problem of using Spark Streaming to accept input data and update > a result. > > The input of the data is from Kafka and the output is to repo

Visualize task distribution in cluster

2014-07-02 Thread Tobias Pfeiffer
Hi, I am using Mesos to run my Spark tasks. I would be interested to see how Spark distributes the tasks in the cluster (nodes, partitions) and which nodes are more or less active and do what kind of tasks, and how long the transfer of data and jobs takes. Is there any way to get this information

Re: Spark job tracker.

2014-07-02 Thread abhiguruvayya
Spark displays job status information on port 4040 using JobProgressListener, any one knows how to hook into this port and read the details? -- View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/Spark-job-tracker-tp8367p8697.html Sent from the Apache Spark User Li

RDD join: composite keys

2014-07-02 Thread Sameer Tilak
Hi everyone, Is it possible to join RDDs using composite keys? I would like to join these two RDDs with RDD1.id1 = RDD2.id1 and RDD1.id2 RDD2.id2RDD1 (id1, id2, scoretype1) RDD2 (id1, id2, scoretype2) I want the result to be ResultRDD = (id1, id2, (score1, score2)) Would really appreciate if you

Enable Parsing Failed or Incompleted jobs on HistoryServer (YARN mode)

2014-07-02 Thread Andrew Lee
Hi All, I have HistoryServer up and running, and it is great. Is it possible to also enable HsitoryServer to parse failed jobs event by default as well? I get "No Completed Applications Found" if job fails. =Event Log Location: hdfs:///user/test01/spark/logs/No Completed Applications Foun

Re: Shark Vs Spark SQL

2014-07-02 Thread 田毅
hi, Matei Do you know how to run the JDBC / Thrift server on Yarn? I did not find any suggestion in docs. 2014-07-02 16:06 GMT-07:00 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-

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: AWS Credentials for private S3 reads

2014-07-02 Thread Brian Gawalt
HUH; not-scrubbing the slashes fixed it. I would have sworn I tried it, got a 403 Forbidden, then remembered the slash prescription. Can confirm I was never scrubbing the actual URIs. It looks like it'd all be working now except it's smacking its head against: 14/07/02 23:37:38 INFO rdd.HadoopRDD:

RE: write event logs with YARN

2014-07-02 Thread Andrew Lee
Hi Christophe, Make sure you have 3 slashes in the hdfs scheme. e.g. hdfs:///:9000/user//spark-events and in the spark-defaults.conf as well.spark.eventLog.dir=hdfs:///:9000/user//spark-events > Date: Thu, 19 Jun 2014 11:18:51 +0200 > From: christophe.pre...@kelkoo.com > To: user@spark.apache.org

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

AWS Credentials for private S3 reads

2014-07-02 Thread Brian Gawalt
Hello everyone, I'm having some difficulty reading from my company's private S3 buckets. I've got an S3 access key and secret key, and I can read the files fine from a non-Spark Scala routine via AWScala. But trying to read them with the SparkContext.textFiles([comma separated s3n://bucket/key u

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

AWS Credentials for private S3 reads

2014-07-02 Thread Brian Gawalt
Hello everyone, I'm having some difficulty reading from my company's private S3 buckets. I've got an S3 access key and secret key, and I can read the files fine from a non-Spark Scala routine via AWScala . But trying to read them with the SparkContext.textFil

Spark Streaming- Input from Kafka, output to HBase

2014-07-02 Thread JiajiaJing
Hi, I am trying to write a program that take input from kafka topics, do some process and write the output to a hbase table. I basically followed the MatricAggregatorHBase example TD created (https://issues.apache.org/jira/browse/SPARK-944), but the problem is that I always get org.apache.hadoop.

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

reduceByKey Not Being Called by Spark Streaming

2014-07-02 Thread Dan H.
Hi all, I recently just picked up Spark and am trying to work through a coding issue that involves the reduceByKey method. After various debugging efforts, it seems that the reducyByKey method never gets called. Here's my workflow, which is followed by my code and results: My parsed data contain

Re: Shark Vs Spark SQL

2014-07-02 Thread Shrikar archak
As of the spark summit 2014 they mentioned that there will be no active development on shark. Thanks, Shrikar On Wed, Jul 2, 2014 at 3:53 PM, Subacini B wrote: > Hi, > > > http://mail-archives.apache.org/mod_mbox/spark-user/201403.mbox/%3cb75376b8-7a57-4161-b604-f919886cf...@gmail.com%3E > > T

Re: Shark Vs Spark SQL

2014-07-02 Thread Chester Chen
Yes, they have announced that Shark is no longer under development and be replaced with Spark SQL in Spark Summit 2014. Chester On Wed, Jul 2, 2014 at 3:53 PM, Subacini B wrote: > Hi, > > > http://mail-archives.apache.org/mod_mbox/spark-user/201403.mbox/%3cb75376b8-7a57-4161-b604-f919886cf...

Shark Vs Spark SQL

2014-07-02 Thread Subacini B
Hi, http://mail-archives.apache.org/mod_mbox/spark-user/201403.mbox/%3cb75376b8-7a57-4161-b604-f919886cf...@gmail.com%3E This talks about Shark backend will be replaced with Spark SQL engine in future. Does that mean Spark will continue to support Shark + Spark SQL for long term? OR After some p

Spark SQL - groupby

2014-07-02 Thread Subacini B
Hi, Below code throws compilation error , "not found: *value Sum*" . Can someone help me on this. Do i need to add any jars or imports ? even for Count , same error is thrown val queryResult = sql("select * from Table) queryResult.groupBy('colA)('colA,*Sum*('colB) as 'totB).aggregate(*Sum*

Use Spark Streaming to update result whenever data come

2014-07-02 Thread Bill Jay
Hi all, I have a problem of using Spark Streaming to accept input data and update a result. The input of the data is from Kafka and the output is to report a map which is updated by historical data in every minute. My current method is to set batch size as 1 minute and use foreachRDD to update th

Kafka - streaming from multiple topics

2014-07-02 Thread Sergey Malov
HI, I would like to set up streaming from Kafka cluster, reading multiple topics and then processing each of the differently. So, I’d create a stream val stream = KafkaUtils.createStream(ssc,"localhost:2181","logs", Map("retarget" -> 2,"datapair" -> 2)) And then based on whether it’s “ret

Re: MLLib : Math on Vector and Matrix

2014-07-02 Thread Dmitriy Lyubimov
in my humble opinion Spark should've supported linalg a-la [1] before it even started dumping methodologies into mllib. [1] http://mahout.apache.org/users/sparkbindings/home.html On Wed, Jul 2, 2014 at 2:16 PM, Thunder Stumpges wrote: > Thanks. I always hate having to do stuff like this. It se

Re: MLLib : Math on Vector and Matrix

2014-07-02 Thread Thunder Stumpges
Thanks. I always hate having to do stuff like this. It seems like they went a bit overboard with all the "private[mllib]" declarations... possibly all in the name of "thou shalt not change your public API". If you don't make your public API usable, we end up having to work around it anyway... Oh w

Re: MLLib : Math on Vector and Matrix

2014-07-02 Thread Koert Kuipers
i did the second option: re-implemented .toBreeze as .breeze using pimp classes On Wed, Jul 2, 2014 at 5:00 PM, Thunder Stumpges wrote: > I am upgrading from Spark 0.9.0 to 1.0 and I had a pretty good amount of > code working with internals of MLLib. One of the big changes was the move > from t

MLLib : Math on Vector and Matrix

2014-07-02 Thread Thunder Stumpges
I am upgrading from Spark 0.9.0 to 1.0 and I had a pretty good amount of code working with internals of MLLib. One of the big changes was the move from the old jblas.Matrix to the Vector/Matrix classes included in MLLib. However I don't see how we're supposed to use them for ANYTHING other than a

LIMIT with offset in SQL queries

2014-07-02 Thread durin
Hi, in many SQL-DBMS like MySQL, you can set an offset for the LIMIT clause, s.t. /LIMIT 5, 10/ will return 10 rows, starting from row 5. As far as I can see, this is not possible in Spark-SQL. The best solution I have to imitate that (using Scala) is converting the RDD into an Array via collect

Re: Run spark unit test on Windows 7

2014-07-02 Thread Denny Lee
You don't actually need it per se - its just that some of the Spark libraries are referencing Hadoop libraries even if they ultimately do not call them. When I was doing some early builds of Spark on Windows, I admittedly had Hadoop on Windows running as well and had not run into this particular is

Re: Run spark unit test on Windows 7

2014-07-02 Thread Kostiantyn Kudriavtsev
No, I don’t why do I need to have HDP installed? I don’t use Hadoop at all and I’d like to read data from local filesystem On Jul 2, 2014, at 9:10 PM, Denny Lee wrote: > By any chance do you have HDP 2.1 installed? you may need to install the > utils and update the env variables per > http:/

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: Run spark unit test on Windows 7

2014-07-02 Thread Denny Lee
By any chance do you have HDP 2.1 installed? you may need to install the utils and update the env variables per http://stackoverflow.com/questions/18630019/running-apache-hadoop-2-1-0-on-windows > On Jul 2, 2014, at 10:20 AM, Konstantin Kudryavtsev > wrote: > > Hi Andrew, > > it's windows 7

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.

NullPointerException on ExternalAppendOnlyMap

2014-07-02 Thread Konstantin Kudryavtsev
Hi all, I catch very confusing exception running Spark 1.0 on HDP2.1 During save rdd as text file I got: 14/07/02 10:11:12 WARN TaskSetManager: Loss was due to java.lang.NullPointerException java.lang.NullPointerException at org.apache.spark.util.collection.ExternalAppendOnlyMap$Externa

Re: Run spark unit test on Windows 7

2014-07-02 Thread Konstantin Kudryavtsev
Hi Andrew, it's windows 7 and I doesn't set up any env variables here The full stack trace: 14/07/02 19:59:31 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable 14/07/02 19:59:31 ERROR Shell: Failed to locate the winutils

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

Spark, Logging Issues: slf4j or log4j

2014-07-02 Thread Shivani Rao
Hello Spark fans, I am unable to figure out how Spark figures out which logger to use. I know that Spark decides upon this at the time of initialization of the Spark Context. From Spark documentation it is clear that Spark uses log4j, and not slf4j, but I have been able to successfully get spark t

Re: spark streaming counter metrics

2014-07-02 Thread Chen Song
Thanks Mayur. I will take a look at StreamingListener. Is there any example you have handy? On Wed, Jul 2, 2014 at 2:32 AM, Mayur Rustagi wrote: > You may be able to mix StreamingListener & SparkListener to get meaningful > information about your task. however you need to connect a lot of piec

Re: Worker can not find custom KryoRegistrator

2014-07-02 Thread Baoxu Shi(Dash)
Don’t know why the setting does not appear in the last mail: .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") .set("spark.kryo.registrator", new HDTMKryoRegistrator().getClass.getName) On Jul 2, 2014, at 1:03 PM, dash wrote: > Hi, I'm using Spark 1.1.0 standal

Worker can not find custom KryoRegistrator

2014-07-02 Thread dash
Hi, I'm using Spark 1.1.0 standalone with 5 workers and 1 driver, and Kryo settings are When I submit this job, the driver works fine but workers will throw ClassNotFoundException saying they can not found HDTMKryoRegistrator. Any idea about this problem? I googled this but there is only one p

Re: SparkKMeans.scala from examples will show: NoClassDefFoundError: breeze/linalg/Vector

2014-07-02 Thread Wanda Hawk
I can run it now with the suggested method. However, I have encountered a new problem that I have not faced before (sent another email with that one but here it goes again ...) I ran SparkKMeans with a big file (~ 7 GB of data) for one iteration with spark-0.8.0 with this line in bash.rc " expo

installing spark 1 on hadoop 1

2014-07-02 Thread Imran Akbar
Hi, I'm trying to install spark 1 on my hadoop cluster running on EMR. I didn't have any problem installing the previous versions, but on this version I couldn't find any 'sbt' folder. However, the README still suggests using this to install Spark: ./sbt/sbt assembly which fails: ./sbt/sbt: N

Run spark unit test on Windows 7

2014-07-02 Thread Konstantin Kudryavtsev
Hi all, I'm trying to run some transformation on *Spark*, it works fine on cluster (YARN, linux machines). However, when I'm trying to run it on local machine (*Windows 7*) under unit test, I got errors: java.io.IOException: Could not locate executable null\bin\winutils.exe in the Hadoop binaries

Re: streaming questions

2014-07-02 Thread mcampbell
Tathagata Das wrote > *Answer 1:*Make sure you are using master as "local[n]" with n > 1 > (assuming you are running it in local mode). The way Spark Streaming > works is that it assigns a code to the data receiver, and so if you > run the program with only one core (i.e., with local or local[1]),

Re: Execution stalls in LogisticRegressionWithSGD

2014-07-02 Thread Bharath Ravi Kumar
Hi Xiangrui, The issue with aggergating/counting over large feature vectors (as part of LogisticRegressionWithSGD) continues to exist, but now in another form: while the execution doesn't freeze (due to SPARK-1112), it now fails at the second or third gradient descent iteration consistently with a

Stream into Parquet Table

2014-07-02 Thread prashant amar
Hi I am a newbie attempting to understand how to use Parquet in cojunction with spark. I am attempting to create a Parquet Table (perhaps partitioned by day) and stream telemetry data (time series) into the parquet table. Are there any examples of InsertIntoParquetTable that I can follow where

[mllib] strange/buggy results with RidgeRegressionWithSGD

2014-07-02 Thread Eustache DIEMERT
Hi list, I'm benchmarking MLlib for a regression task [1] and get strange results. Namely, using RidgeRegressionWithSGD it seems the predicted points miss the intercept: {code} val trainedModel = RidgeRegressionWithSGD.train(trainingData, 1000) ... valuesAndPreds.take(10).map(t => println(t)) {c

Re: SparkKMeans.scala from examples will show: NoClassDefFoundError: breeze/linalg/Vector

2014-07-02 Thread Yana Kadiyska
The scripts that Xiangrui mentions set up the classpath...Can you run ./run-example for the provided example sucessfully? What you can try is set SPARK_PRINT_LAUNCH_COMMAND=1 and then call run-example -- that will show you the exact java command used to run the example at the start of execution. A

Re: How to terminate job from the task code?

2014-07-02 Thread Piotr Kołaczkowski
SparkContext is not serializable and can't be just "sent across" ;) 2014-06-21 14:14 GMT+02:00 Mayur Rustagi : > You can terminate job group from spark context, Youll have to send across > the spark context to your task. > On 21 Jun 2014 01:09, "Piotr Kołaczkowski" wrote: > >> If the task det

Re: Help alleviating OOM errors

2014-07-02 Thread Yana Kadiyska
Can you elaborate why "You need to configure the spark.shuffle.spill true again in the config" -- the default for spark.shuffle.spill is set to true according to the doc(https://spark.apache.org/docs/0.9.1/configuration.html)? On OOM the tasks were process_local, which I understand is "as good as

Custom Serialization

2014-07-02 Thread Andrea Esposito
Hi, i have a non-serializable class and as workaround i'm trying to re-instantiate it at each de-serialization. Thus, i created a wrapper class and overridden the writeObject and readObject methods as follow: > private def writeObject(oos: ObjectOutputStream) { > oos.defaultWriteObject() >

java.io.FileNotFoundException: shuffle

2014-07-02 Thread nit
Hi, I am running my spark job on Yarn; using latest code from master branch..synced few days back. I see this IO Exception during shuffle(in resource manager logs). What could be wrong and how to debug it? I have seen this few times before; I was suspecting that this could side effect of memory pr

java options for spark-1.0.0

2014-07-02 Thread Wanda Hawk
I ran SparkKMeans with a big file (~ 7 GB of data) for one iteration with spark-0.8.0 with this line in bash.rc " export _JAVA_OPTIONS="-Xmx15g -Xms15g -verbose:gc -XX:+PrintGCTimeStamps -XX:+PrintGCDetails" ". It finished in a decent time, ~50 seconds, and I had only a few "Full GC" message

How to use groupByKey and CqlPagingInputFormat

2014-07-02 Thread Martin Gammelsæter
Hi! Total Scala and Spark noob here with a few questions. I am trying to modify a few of the examples in the spark repo to fit my needs, but running into a few problems. I am making an RDD from Cassandra, which I've finally gotten to work, and trying to do some operations on it. Specifically I a

Re: Where to set proxy in order to run ./install-dev.sh for SparkR

2014-07-02 Thread Ye Xianjin
You can try setting your HTTP_PROXY environment variable. export HTTP_PROXY=host:port But I don't use maven. If the env variable doesn't work, please search google for maven proxy. I am sure there will be a lot of related results. Sent from my iPhone > On 2014年7月2日, at 19:04, Stuti Awasthi wr

Where to set proxy in order to run ./install-dev.sh for SparkR

2014-07-02 Thread Stuti Awasthi
Hi, I wanted to build SparkR from source but running the script behind the proxy. Where shall I set proxy host and port in order to build the source. Issue is not able to download dependencies from Maven Thanks Stuti Awasthi ::DISCLAIMER::

Re: org.jboss.netty.channel.ChannelException: Failed to bind to: master/1xx.xx..xx:0

2014-07-02 Thread MEETHU MATHEW
The problem is resolved.I have added SPARK_LOCAL_IP=master in both slaves also.When i changed this my slaves are working. Thank you all for your suggestions   Thanks & Regards, Meethu M On Wednesday, 2 July 2014 10:22 AM, Aaron Davidson wrote: In your spark-env.sh, do you happen to set SPA

RE: Help: WARN AbstractNioSelector: Unexpected exception in the selector loop. java.lang.OutOfMemoryError: Java heap space

2014-07-02 Thread innowireless TaeYun Kim
It seems that the driver program gets out of memory. In Windows Task Manager, the driver program's memory constantly grows until around 3,434,796, then java OutOfMemory exception occurs. (BTW, the driver program runs on Windows 7 64bit machine, and cluster is on CentOS.) Why the memory of the driv

RE: Help: WARN AbstractNioSelector: Unexpected exception in the selector loop. java.lang.OutOfMemoryError: Java heap space

2014-07-02 Thread innowireless TaeYun Kim
Also, the machine on which the driver program runs constantly uses about 7~8% of 100Mbps network connection. Is the driver program involved in the reduceByKey() somehow? BTW, currently an accumulator is used, but the network usage does not drop even when accumulator is removed. Thanks in advance.

Is it possible to run HiveThriftServer2 based on SparkSQL in YARN now?

2014-07-02 Thread 田毅
Hi, everyone! Is it possible to run HiveThriftServer2 based on SparkSQL in YARN now? Spark version: branch 1.0-jdbc YARN version: 2.3.0-cdh5.0.0

Help: WARN AbstractNioSelector: Unexpected exception in the selector loop. java.lang.OutOfMemoryError: Java heap space

2014-07-02 Thread innowireless TaeYun Kim
Hi, When running a Spark job, the following warning message displays and the job seems no longer progressing. (Detailed log message are at the bottom of this message.) --- 14/07/02 17:00:14 WARN AbstractNioSelector: Unexpected exception in the selector loop. java.lang.OutOfMemoryError

Re: Serializer or Out-of-Memory issues?

2014-07-02 Thread Mayur Rustagi
Your executors are going out of memory & then subsequent tasks scheduled on the scheduler are also failing, hence the lost tid(task id). Mayur Rustagi Ph: +1 (760) 203 3257 http://www.sigmoidanalytics.com @mayur_rustagi On Mon, Jun 30, 2014 at 7:47 PM, Sguj

Configure and run external process with RDD.pipe

2014-07-02 Thread Jaonary Rabarisoa
Hi all, I need to run a complex external process with a lots of dependencies from spark. The "pipe" and "addFile" function seem to be my friends but there are just some issues that I need to solve. Precisely, the process I want to run are C++ executable that may depend on some libraries and addit

Re: Error: UnionPartition cannot be cast to org.apache.spark.rdd.HadoopPartition

2014-07-02 Thread Mayur Rustagi
two job context cannot share data, are you collecting the data to the master & then sending it to the other context? Mayur Rustagi Ph: +1 (760) 203 3257 http://www.sigmoidanalytics.com @mayur_rustagi On Wed, Jul 2, 2014 at 11:57 AM, Honey Joshi < honeyjo...@i