Re: Spark will process _temporary folder on S3 is very slow and always cause failure

2015-03-17 Thread Imran Rashid
I'm not super familiar w/ S3, but I think the issue is that you want to use a different output committers with object stores, that don't have a simple move operation. There have been a few other threads on S3 outputcommitters. I think the most relevant for you is most probably this open JIRA:

Re: Need Advice about reading lots of text files

2015-03-17 Thread Imran Rashid
Interesting, on another thread, I was just arguing that the user should *not* open the files themselves and read them, b/c then they lose all the other goodies we have in HadoopRDD, eg. the metric tracking. I think this encourages Pat's argument that we might actually need better support for this

HIVE SparkSQL

2015-03-17 Thread 宫勐
Hi: I need to migrate a Log Analysis System from mysql + some C++ real time computer framwork to Hadoop ecosystem. When I want to build a data warehouse. don't know which one is the right choice. Cassandra? HIVE? Or just SparkSQL ? There is few benchmark for these systems. My

Re: HDP 2.2 AM abort : Unable to find ExecutorLauncher class

2015-03-17 Thread Bharath Ravi Kumar
Hi Todd, Yes, those entries were present in the conf under the same SPARK_HOME that was used to run spark-submit. On a related note, I'm assuming that the additional spark yarn options (like spark.yarn.jar) need to be set in the same properties file that is passed to spark-submit. That apart, I

Re: Apache Spark Executor - number of threads

2015-03-17 Thread nirandap
Hi devs, I would like to know this as well. It would be great if someone could provide this information. cheers On Tue, Mar 17, 2015 at 3:06 PM, Igor Petrov [via Apache Spark User List] ml-node+s1001560n22095...@n3.nabble.com wrote: Hello, is it possible to set number of threads in the

Re: What is best way to run spark job in yarn-cluster mode from java program(servlet container) and NOT using spark-submit command.

2015-03-17 Thread Akhil Das
Create SparkContext set master as yarn-cluster then run it as a standalone program? Thanks Best Regards On Tue, Mar 17, 2015 at 1:27 AM, rrussell25 rrussel...@gmail.com wrote: Hi, were you ever able to determine a satisfactory approach for this problem? I have a similar situation and would

Re: Hive on Spark with Spark as a service on CDH5.2

2015-03-17 Thread Arush Kharbanda
Hive on Spark and accessing HiveContext from the shall are seperate things. Hive on Spark - https://cwiki.apache.org/confluence/display/Hive/Hive+on+Spark%3A+Getting+Started To access hive on Spark you need to built with -Phive.

Re: Spark @ EC2: Futures timed out Ask timed out

2015-03-17 Thread Akhil Das
Did you launch the cluster using spark-ec2 script? Just make sure all ports are open for master, slave instances security group. From the error, it seems its not able to connect to the driver program (port 58360) Thanks Best Regards On Tue, Mar 17, 2015 at 3:26 AM, Otis Gospodnetic

Re: Any IRC channel on Spark?

2015-03-17 Thread Ahmed Nawar
Dears, Is there any instructions to build spark 1.3.0 on windows 7. I tried mvn -Phive -Phive-thriftserver -DskipTests clean package but i got below errors [INFO] Spark Project Parent POM ... SUCCESS [ 7.845 s] [INFO] Spark Project Networking

build spark 1.3.0 on windows 7.

2015-03-17 Thread Ahmed Nawar
Sorry for old subject i am correcting it. On Tue, Mar 17, 2015 at 11:47 AM, Ahmed Nawar ahmed.na...@gmail.com wrote: Dears, Is there any instructions to build spark 1.3.0 on windows 7. I tried mvn -Phive -Phive-thriftserver -DskipTests clean package but i got below errors [INFO]

Re: Priority queue in spark

2015-03-17 Thread twinkle sachdeva
In that case, having pre configured pools, but using the correct pool at code level might do. On Tue, Mar 17, 2015 at 11:23 AM, abhi abhishek...@gmail.com wrote: yes . Each generated job can have a different priority it is like a recursive function, where in each iteration generate job will

Re: MappedStream vs Transform API

2015-03-17 Thread madhu phatak
Hi, Thank you for the response. Can I give a PR to use transform for all the functions like map,flatMap etc so they are consistent with other API's?. Regards, Madhukara Phatak http://datamantra.io/ On Mon, Mar 16, 2015 at 11:42 PM, Tathagata Das t...@databricks.com wrote: It's mostly for

Re: why generateJob is a private API?

2015-03-17 Thread madhu phatak
Hi, Thank you for the response. Regards, Madhukara Phatak http://datamantra.io/ On Tue, Mar 17, 2015 at 5:50 AM, Tathagata Das t...@databricks.com wrote: It was not really meant to be pubic and overridden. Because anything you want to do to generate jobs from RDDs can be done using

Re: Upgrade from Spark 1.1.0 to 1.1.1+ Issues

2015-03-17 Thread Akhil Das
Could you tell me what all you did to change the version of spark? Can you fireup a spark-shell and write this line and see what happens: sc.parallelize(1 to 1).collect() Thanks Best Regards On Mon, Mar 16, 2015 at 11:13 PM, Eason Hu eas...@gmail.com wrote: Hi Akhil, Yes, I did change

Hive on Spark with Spark as a service on CDH5.2

2015-03-17 Thread anu
*I am not clear if spark sql supports HIve on Spark when spark is run as a service in CDH 5.2? * Can someone please clarify this. If this is possible, how what configuration changes have I to make to import hive context in spark shell as well as to be able to do a spark-submit for the job to be

Re: Any IRC channel on Spark?

2015-03-17 Thread Akhil Das
There's one on Freenode, You can join #Apache-Spark There's like 60 people idling. :) Thanks Best Regards On Mon, Mar 16, 2015 at 10:46 PM, Feng Lin lfliu.x...@gmail.com wrote: Hi, everyone, I'm wondering whether there is a possibility to setup an official IRC channel on freenode. I

Re: HiveContext can't find registered function

2015-03-17 Thread Yin Huai
Initially, an attribute reference (column reference), like selecting a column from a table, is not resolved since we do not know if the reference is valid or not (if this column exists in the underlying table). In the query compilation process, we will first analyze this query and resolved those

graceful shutdown not so graceful?

2015-03-17 Thread necro351 .
Hi all, I am trying to do a graceful shutdown of my spark streaming job and it appears that everything shuts down gracefully but the checkpointing thread, which continues to run until it crashes. I looked at the checkpoint thread in 1.3.0 (

RE: Date and decimal datatype not working

2015-03-17 Thread BASAK, ANANDA
Ok, thanks for the suggestions. Let me try and will confirm all. Regards Ananda From: Yin Huai [mailto:yh...@databricks.com] Sent: Tuesday, March 17, 2015 3:04 PM To: BASAK, ANANDA Cc: user@spark.apache.org Subject: Re: Date and decimal datatype not working p(0) is a String. So, you need to

shuffle write size

2015-03-17 Thread Chen Song
I have a map reduce job that reads from three logs and joins them on some key column. The underlying data is protobuf messages in sequence files. Between mappers and reducers, the underlying raw byte arrays for protobuf messages are shuffled . Roughly, for 1G input from HDFS, there is 2G data

Date and decimal datatype not working

2015-03-17 Thread BASAK, ANANDA
Hi All, I am very new in Spark world. Just started some test coding from last week. I am using spark-1.2.1-bin-hadoop2.4 and scala coding. I am having issues while using Date and decimal data types. Following is my code that I am simply running on scala prompt. I am trying to define a table and

Re: Date and decimal datatype not working

2015-03-17 Thread Yin Huai
p(0) is a String. So, you need to explicitly convert it to a Long. e.g. p(0).trim.toLong. You also need to do it for p(2). For those BigDecimals value, you need to create BigDecimal objects from your String values. On Tue, Mar 17, 2015 at 5:55 PM, BASAK, ANANDA ab9...@att.com wrote: Hi All,

Apache Spark Executor - number of threads

2015-03-17 Thread Igor Petrov
Hello, is it possible to set number of threads in the Executor's pool? I see no such setting in the docs. The reason we want to try it: we want to see performance impact with different level of parallelism (having one thread per CPU, two threads per CPU, N threads per CPU). Thank You -- View

TreeNodeException: Unresolved plan found

2015-03-17 Thread Ophir Cohen
Hi Guys and great job! I encounter a weird problem on local mode and I'll be glad to solve it out... When trying to save ScehmaRDD into Hive table it fails with 'TreeNodeException: Unresolved plan found' I have found similar issue in Jira: https://issues.apache.org/jira/browse/SPARK-4825 but I'm

Spark-submit and multiple files

2015-03-17 Thread poiuytrez
Hello guys, I am having a hard time to understand how spark-submit behave with multiple files. I have created two code snippets. Each code snippet is composed of a main.py and work.py. The code works if I paste work.py then main.py in a pyspark shell. However both snippets do not work when using

Re: Unable to find org.apache.spark.sql.catalyst.ScalaReflection class

2015-03-17 Thread Eric Charles
Launching from eclipse (scala-ide) as a scala process gives such error, but as a java process (a java main class) works fine. Launching as a scala process from Intellij works fine. There is something wrong at eclipse side, not in Spark. On 03/13/2015 11:47 AM, Jianshi Huang wrote: Liancheng

Re: MappedStream vs Transform API

2015-03-17 Thread madhu phatak
Hi, Regards, Madhukara Phatak http://datamantra.io/ On Tue, Mar 17, 2015 at 2:31 PM, Tathagata Das t...@databricks.com wrote: That's not super essential, and hence hasn't been done till now. Even in core Spark there are MappedRDD, etc. even though all of them can be implemented by

Re: ClassNotFoundException

2015-03-17 Thread Ralph Bergmann
Hi Kevin, yes I can test it means I have to build Spark from git repository? Ralph Am 17.03.15 um 02:59 schrieb Kevin (Sangwoo) Kim: Hi Ralph, It seems like https://issues.apache.org/jira/browse/SPARK-6299 issue, which is I'm working on. I submitted a PR for it, would you test it?

Hive error on partitioned tables

2015-03-17 Thread Masf
Hi. I'm running Spark 1.2.0. I have HiveContext and I execute the following query: select sum(field1 / 100) from table1 group by field2; field1 in hive metastore is a smallint. The schema detected by hivecontext is a int32: fileSchema: message schema { optional int32 field1;

Building Spark on Windows WAS: Any IRC channel on Spark?

2015-03-17 Thread Ted Yu
Have you tried with -X switch ? Thanks On Mar 17, 2015, at 1:47 AM, Ahmed Nawar ahmed.na...@gmail.com wrote: Dears, Is there any instructions to build spark 1.3.0 on windows 7. I tried mvn -Phive -Phive-thriftserver -DskipTests clean package but i got below errors

IllegalAccessError in GraphX (Spark 1.3.0 LDA)

2015-03-17 Thread Jeffrey Jedele
Hi all, I'm trying to use the new LDA in mllib, but when trying to train the model, I'm getting following error: java.lang.IllegalAccessError: tried to access class org.apache.spark.util.collection.Sorter from class org.apache.spark.graphx.impl.EdgePartitionBuilder at

Re: MappedStream vs Transform API

2015-03-17 Thread Tathagata Das
That's not super essential, and hence hasn't been done till now. Even in core Spark there are MappedRDD, etc. even though all of them can be implemented by MapPartitionedRDD (may be the name is wrong). So its nice to maintain the consistency, MappedDStream creates MappedRDDs. :) Though this does

Re: Building Spark on Windows WAS: Any IRC channel on Spark?

2015-03-17 Thread Ahmed Nawar
Dear Yu, With -X i got below error. [INFO] [INFO] Reactor Summary: [INFO] [INFO] Spark Project Parent POM ... SUCCESS [ 7.418 s] [INFO] Spark Project Networking ...

Should I do spark-sql query on HDFS or apache hive?

2015-03-17 Thread 李铖
Hi,everybody. I am new in spark. Now I want to do interactive sql query using spark sql. spark sql can run under hive or loading files from hdfs. Which is better or faster? Thanks.

Re: MappedStream vs Transform API

2015-03-17 Thread madhu phatak
Hi, Sorry for the wrong formatting in the earlier mail. On Tue, Mar 17, 2015 at 2:31 PM, Tathagata Das t...@databricks.com wrote: That's not super essential, and hence hasn't been done till now. Even in core Spark there are MappedRDD, etc. even though all of them can be implemented by

Should I do spark-sql query on HDFS or hive?

2015-03-17 Thread 李铖
Hi,everybody. I am new in spark. Now I want to do interactive sql query using spark sql. spark sql can run under hive or loading files from hdfs. Which is better or faster? Thanks.

Re: Spark SQL UDT Kryo serialization, Unable to find class

2015-03-17 Thread Michael Armbrust
I'll caution you that this is not a stable public API. That said, it seems that the issue is that you have not copied the jar file containing your class to all of the executors. You should not need to do any special configuration of serialization (you can't for SQL, as we hard code it for

Re: Need Advice about reading lots of text files

2015-03-17 Thread Michael Armbrust
I agree that it would be better if Spark did a better job automatically here, though doing so is probably a non-trivial amount of work. My code is certainly worse if you have only a few very large text files for example and thus I'd generally encourage people to try the built in options first.

Re: High GC time

2015-03-17 Thread Xiangrui Meng
The official guide may help: http://spark.apache.org/docs/latest/tuning.html#garbage-collection-tuning -Xiangrui On Tue, Mar 17, 2015 at 8:27 AM, jatinpreet jatinpr...@gmail.com wrote: Hi, I am getting very high GC time in my jobs. For smaller/real-time load, this becomes a real problem.

Re: Should I do spark-sql query on HDFS or apache hive?

2015-03-17 Thread Michael Armbrust
The performance has more to do with the particular format you are using, not where the metadata is coming from. Even hive tables are read from files HDFS usually. You probably should use HiveContext as its query language is more powerful than SQLContext. Also, parquet is usually the faster

Re: HiveContext can't find registered function

2015-03-17 Thread Ophir Cohen
Interesting, I thought the problem is with the method itself. I will check it soon and update. Can you elaborate what does it mean the # and the number? Is that a reference to the field in the rdd? Thank you, Ophir On Mar 17, 2015 7:06 PM, Yin Huai yh...@databricks.com wrote: Seems

Re: TreeNodeException: Unresolved plan found

2015-03-17 Thread Ophir Cohen
Ok, I managed to solve it. As the issue in jira suggests it fixed in 1.2.1, i probably had some old jars in the classpath. Cleaning everything and rebuild eventually solve the problem. On Mar 17, 2015 12:25 PM, Ophir Cohen oph...@gmail.com wrote: Hi Guys and great job! I encounter a weird

Re: RDD to DataFrame for using ALS under org.apache.spark.ml.recommendation.ALS

2015-03-17 Thread Xiangrui Meng
Please check this section in the user guide: http://spark.apache.org/docs/latest/sql-programming-guide.html#inferring-the-schema-using-reflection You need `import sqlContext.implicits._` to use `toDF()`. -Xiangrui On Mon, Mar 16, 2015 at 2:34 PM, Jay Katukuri jkatuk...@apple.com wrote: Hi

Re: Garbage stats in Random Forest leaf node?

2015-03-17 Thread Xiangrui Meng
This is the default value (Double.MinValue) for invalid gain: https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala#L67 Please ignore it. Maybe we should update `toString` to use scientific notation. -Xiangrui On Mon, Mar

Re: Need Advice about reading lots of text files

2015-03-17 Thread Pat Ferrel
There are no-doubt many things that feed into the right way to read a lot of files into Spark. But why force users to learn all of those factors instead of putting an optimizer layer into the read inside Spark? BTW I realize your method is not one task per file, it’s chunked and done in

Re: HiveContext can't find registered function

2015-03-17 Thread Yin Huai
The number is an id we used internally to identify an resolved Attribute. Looks like basic_null_diluted_d was not resolved since there is no id associated with it. On Tue, Mar 17, 2015 at 2:08 PM, Ophir Cohen oph...@gmail.com wrote: Interesting, I thought the problem is with the method itself.

Set spark.fileserver.uri on private cluster

2015-03-17 Thread Rares Vernica
Hi, I have a private cluster with private IPs, 192.168.*.*, and a gateway node with both private IP, 192.168.*.*, and public internet IP. I setup the Spark master on the gateway node and set the SPARK_MASTER_IP to the private IP. I start Spark workers on the private nodes. It works fine. The

Re: Spark will process _temporary folder on S3 is very slow and always cause failure

2015-03-17 Thread Aaron Davidson
Actually, this is the more relevant JIRA (which is resolved): https://issues.apache.org/jira/browse/SPARK-3595 6352 is about saveAsParquetFile, which is not in use here. Here is a DirectOutputCommitter implementation: https://gist.github.com/aarondav/c513916e72101bbe14ec and it can be

saveAsTable fails to save RDD in Spark SQL 1.3.0

2015-03-17 Thread smoradi
Hi, Basically my goal is to make the Spark SQL RDDs available to Tableau software through Simba ODBC driver. I’m running standalone Spark 1.3.0 on Ubuntu 14.04. Got the source code and complied it with maven. Hive is also setup and connected to mysql all on a the same machine. The hive-site.xml

Re: Question on Spark 1.3 SQL External Datasource

2015-03-17 Thread Yang Lei
Thanks Cheng for the clarification. Looking forward to this new API mentioned below. Yang Sent from my iPad On Mar 17, 2015, at 8:05 PM, Cheng Lian lian.cs@gmail.com wrote: Hey Yang, My comments are in-lined below. Cheng On 3/18/15 6:53 AM, Yang Lei wrote: Hello, I am

Re: InvalidAuxServiceException in dynamicAllocation

2015-03-17 Thread Marcelo Vanzin
I assume you're running YARN given the exception. I don't know if this is covered in the documentation (I took a quick look at the config document and didn't see references to it), but you need to configure Spark's external shuffle service as and auxiliary nodemanager service in your YARN

Re: StorageLevel: OFF_HEAP

2015-03-17 Thread Ted Yu
Ranga: Take a look at https://github.com/apache/spark/pull/4867 Cheers On Tue, Mar 17, 2015 at 6:08 PM, fightf...@163.com fightf...@163.com wrote: Hi, Ranga That's true. Typically a version mis-match issue. Note that spark 1.2.1 has tachyon built in with version 0.5.0 , I think you may need

Re: Downloading data from url

2015-03-17 Thread Ted Yu
The methods I mentioned are private. But I hope they give you some idea how downloading from url works. Cheers On Tue, Mar 17, 2015 at 7:01 AM, Ted Yu yuzhih...@gmail.com wrote: Please take a look at downloadFile() method in ./core/src/main/scala/org/apache/spark/util/Utils.scala You can

InvalidAuxServiceException in dynamicAllocation

2015-03-17 Thread Sea
Hi, all: Spark1.3.0 hadoop2.2.0 I put the following params in the spark-defaults.conf spark.dynamicAllocation.enabled true spark.dynamicAllocation.minExecutors 20 spark.dynamicAllocation.maxExecutors 300 spark.dynamicAllocation.executorIdleTimeout 300 spark.shuffle.service.enabled true‍

StorageLevel: OFF_HEAP

2015-03-17 Thread Ranga
Hi I am trying to use the OFF_HEAP storage level in my Spark (1.2.1) cluster. The Tachyon (0.6.0-SNAPSHOT) nodes seem to be up and running. However, when I try to persist the RDD, I get the following error: ERROR [2015-03-16 22:22:54,017] ({Executor task launch worker-0}

Question on Spark 1.3 SQL External Datasource

2015-03-17 Thread Yang Lei
Hello, I am migrating my Spark SQL external datasource integration from Spark 1.2.x to Spark 1.3. I noticed, there are a couple of new filters now, e.g. org.apache.spark.sql.sources.And. However, for a sql with condition A AND B, I noticed PrunedFilteredScan.buildScan still gets an

Re: Should I do spark-sql query on HDFS or apache hive?

2015-03-17 Thread Michael Armbrust
I am trying to explain that these are not either/or decisions. You are likely going to be storing the data on HDFS no matter what other choices you make. You can use parquet to store the data whether or not you are addressing files directly on HDFS or using the Hive Metastore to locate the

Idempotent count

2015-03-17 Thread Binh Nguyen Van
Hi all, I am new to Spark so please forgive me if my questions is stupid. I am trying to use Spark-Streaming in an application that read data from a queue (Kafka) and do some aggregation (sum, count..) and then persist result to an external storage system (MySQL, VoltDB...) From my understanding

Re: 1.3 release

2015-03-17 Thread Eric Friedman
Yes, I did, with these arguments: --tgz -Pyarn -Phadoop-2.4 -Phive -Phive-thriftserver To be more specific about what is not working, when I launch spark-shell --master yarn, I get this error immediately after launch. I have no idea from looking at the source. java.lang.NullPointerException at

Re: Upgrade from Spark 1.1.0 to 1.1.1+ Issues

2015-03-17 Thread Eason Hu
Hi Akhil, sc.parallelize(1 to 1).collect() in the Spark shell on Spark v1.2.0 runs fine. However, if I do the following remotely, it will throw exception: val sc : SparkContext = new SparkContext(conf) val NUM_SAMPLES = 10 val count = sc.parallelize(1 to NUM_SAMPLES).map{i = val x

Re: Can LBFGS be used on streaming data?

2015-03-17 Thread EcoMotto Inc.
Hello Jeremy, Thank you for your reply. When I am running this code on the local machine on a streaming data, it keeps giving me this error: *WARN TaskSetManager: Lost task 2.0 in stage 211.0 (TID 4138, localhost): java.io.FileNotFoundException:

Re: Question on Spark 1.3 SQL External Datasource

2015-03-17 Thread Cheng Lian
Hey Yang, My comments are in-lined below. Cheng On 3/18/15 6:53 AM, Yang Lei wrote: Hello, I am migrating my Spark SQL external datasource integration from Spark 1.2.x to Spark 1.3. I noticed, there are a couple of new filters now, e.g. org.apache.spark.sql.sources.And. However, for a

Using Spark with a SOCKS proxy

2015-03-17 Thread Kelly, Jonathan
I'm trying to figure out how I might be able to use Spark with a SOCKS proxy. That is, my dream is to be able to write code in my IDE then run it without much trouble on a remote cluster, accessible only via a SOCKS proxy between the local development machine and the master node of the cluster

ML Pipeline question about caching

2015-03-17 Thread Cesar Flores
Hello all: I am using the ML Pipeline, which I consider very powerful. I have the next use case: - I have three transformers, which I will call A,B,C, that basically extract features from text files, with no parameters. - I have a final stage D, which is the logistic regression

Re: ML Pipeline question about caching

2015-03-17 Thread Peter Rudenko
Hi Cesar, I had a similar issue. Yes for now it’s better to do A,B,C outside a crossvalidator. Take a look to my comment https://issues.apache.org/jira/browse/SPARK-4766?focusedCommentId=14320038page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14320038 and this

Re: Hanging tasks in spark 1.2.1 while working with 1.1.1

2015-03-17 Thread Eugen Cepoi
Doing the reduceByKey without changing the number of partitions and then do a coalesce works. But the other version still hangs, without any information (while working with spark 1.1.1). The previous logs don't seem to be related to what happens. I don't think this is a memory issue as the GC time

Memory Settings for local execution context

2015-03-17 Thread Alex Turner (TMS)
So the page that talks about settings: http://spark.apache.org/docs/1.2.1/configuration.html seems to not apply when running local contexts. I have a shell script that starts my job: xport SPARK_MASTER_OPTS=-Dsun.io.serialization.extendedDebugInfo=true export

Re: Should I do spark-sql query on HDFS or apache hive?

2015-03-17 Thread 李铖
Did you mean that parquet is faster than hive format ,and hive format is faster than hdfs ,for Spark SQL? : ) 2015-03-18 1:23 GMT+08:00 Michael Armbrust mich...@databricks.com: The performance has more to do with the particular format you are using, not where the metadata is coming from.

Re: Hanging tasks in spark 1.2.1 while working with 1.1.1

2015-03-17 Thread Dmitriy Lyubimov
FWIW observed similar behavior in similar situation. Was able to work around by forcefully committing one of the rdds right before the union into cache, and forcing that by executing take(1). Nothing else ever helped. Seems like yet-undiscovered 1.2.x thing. On Tue, Mar 17, 2015 at 4:21 PM,

Question on RDD groupBy and executors

2015-03-17 Thread Vijayasarathy Kannan
Hi, I am doing a groupBy on an EdgeRDD like this, val groupedEdges = graph.edges.groupBy[VertexId](func0) while(true) { val info = groupedEdges.flatMap(func1).collect.foreach(func2) } The groupBy distributes the data to different executors on different nodes in the cluster. Given a key K (a

Log4j files per spark job

2015-03-17 Thread Dan H.
Hey guys, Looking for a bit of help on logging. I trying to get Spark to write log4j logs per job within a Spark cluster. So for example, I'd like: $SPARK_HOME/logs/job1.log.x $SPARK_HOME/logs/job2.log.x And I want this on the driver and on the executor. I'm trying to accomplish this by using

Re: HiveContext can't find registered function

2015-03-17 Thread Ophir Cohen
Thanks you for the answer and one more question: what does it mean 'resolved attribute'? On Mar 17, 2015 8:14 PM, Yin Huai yh...@databricks.com wrote: The number is an id we used internally to identify an resolved Attribute. Looks like basic_null_diluted_d was not resolved since there is no id

Re: IllegalAccessError in GraphX (Spark 1.3.0 LDA)

2015-03-17 Thread Xiangrui Meng
Please check your classpath and make sure you don't have multiple Spark versions deployed. If the classpath looks correct, please create a JIRA for this issue. Thanks! -Xiangrui On Tue, Mar 17, 2015 at 2:03 AM, Jeffrey Jedele jeffrey.jed...@gmail.com wrote: Hi all, I'm trying to use the new LDA

Using regular rdd transforms on schemaRDD

2015-03-17 Thread kpeng1
Hi All, I was wondering how rdd transformation work on schemaRDDs. Is there a way to force the rdd transform to keep the schemaRDD types or do I need to recreate the schemaRDD by applying the applySchema method? Currently what I have is an array of SchemaRDDs and I just want to do a union

Spark 1.0.2 failover doesnt port running application context to new master

2015-03-17 Thread Nirav Patel
We have spark 1.0.2 cluster with 3 nodes under HA setup using zookeeper. We have long running self contained spark service that serves on-demand requests. I tried to do failover test by killing spark master and see if our application get ported over to new master. Looks like killing master doesn't

Re: RDD to DataFrame for using ALS under org.apache.spark.ml.recommendation.ALS

2015-03-17 Thread Xiangrui Meng
Please remember to copy the user list next time. I might not be able to respond quickly. There are many others who can help or who can benefit from the discussion. Thanks! -Xiangrui On Tue, Mar 17, 2015 at 12:04 PM, Jay Katukuri jkatuk...@apple.com wrote: Great Xiangrui. It works now. Sorry

Re: Garbage stats in Random Forest leaf node?

2015-03-17 Thread Xiangrui Meng
There are two cases: minInstancesPerNode not satisfied or minInfoGain not satisfied: https://github.com/apache/spark/blob/9b746f380869b54d673e3758ca5e4475f76c864a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala#L729

Re: IllegalAccessError in GraphX (Spark 1.3.0 LDA)

2015-03-17 Thread Jeffrey Jedele
Hi Xiangrui, thank you a lot for the hint! I just tried on another machine with a clean project and there it worked like a charm. Will retry on the other machine tomorrow. Regards, Jeff 2015-03-17 19:57 GMT+01:00 Xiangrui Meng men...@gmail.com: Please check your classpath and make sure you

Re: Using regular rdd transforms on schemaRDD

2015-03-17 Thread kpeng1
Looks like if I use unionAll this works. -- View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/Using-regular-rdd-transforms-on-schemaRDD-tp22105p22107.html Sent from the Apache Spark User List mailing list archive at Nabble.com.

Spark SQL UDT Kryo serialization, Unable to find class

2015-03-17 Thread zia_kayani
Hi, I want to introduce custom type for SchemaRDD, I'm following this https://github.com/apache/spark/blob/branch-1.2/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala example. But I'm having Kryo Serialization issues, here is stack trace:

Re: HDP 2.2 AM abort : Unable to find ExecutorLauncher class

2015-03-17 Thread Todd Nist
Hi Bharath, Do you have these entries in your $SPARK_HOME/conf/spark-defaults.conf file? spark.driver.extraJavaOptions -Dhdp.version=2.2.0.0-2041 spark.yarn.am.extraJavaOptions -Dhdp.version=2.2.0.0-2041 On Tue, Mar 17, 2015 at 1:04 AM, Bharath Ravi Kumar reachb...@gmail.com wrote: Still

GraphX - Correct path traversal order from an Array[Edge[ED]]

2015-03-17 Thread bertlhf
Below is a listing from a filtered Array[Edge[ED]]: scala altGraph.edges.filter {case (edge) = edge.attr.contains(wien-krak-s103) }.collect.foreach { case (edge) = println(sSrcId = ${edge.srcId}, DstId = ${edge.dstId})} SrcId = 1, DstId = 2

Re: Building Spark on Windows WAS: Any IRC channel on Spark?

2015-03-17 Thread Ted Yu
Can you look in build output for scalastyle warning in mllib module ? Cheers On Mar 17, 2015, at 3:00 AM, Ahmed Nawar ahmed.na...@gmail.com wrote: Dear Yu, With -X i got below error. [INFO] [INFO]

Re: Building Spark on Windows WAS: Any IRC channel on Spark?

2015-03-17 Thread Ahmed Nawar
Dear Yu, Are you mean scalastyle-output.xml? i coped its content below ?xml version=1.0 encoding=UTF-8? checkstyle version=5.0 file name=C:\Nawwar\Hadoop\spark\spark-1.3.0\mllib\src\main\scala\org\apache\spark\mllib\clustering\LDAModel.scala error severity=error message=Input length =

Re: Process time series RDD after sortByKey

2015-03-17 Thread Shawn Zheng
Hi Imran, This is extremely helpful. This is not only an approach, also help me to understand how to affect or customize my own DAG effectively. Thanks a lot! Shuai On Monday, March 16, 2015, Imran Rashid iras...@cloudera.com wrote: Hi Shuai, yup, that is exactly what I meant -- implement

HiveContext can't find registered function

2015-03-17 Thread Ophir Cohen
Hi Guys, I'm registering a function using: sqlc.registerFunction(makeEstEntry,ReutersDataFunctions.makeEstEntry _) Then I register the table and try to query the table using that function and I get: org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Unresolved attributes:

Downloading data from url

2015-03-17 Thread Hafiz Mujadid
Hi experts! Is there any api in spark to download data from url? I want to download data from url in a spark application. I want to get downloading on all nodes instead of a single node. Thanks -- View this message in context:

Re: Downloading data from url

2015-03-17 Thread Ted Yu
Please take a look at downloadFile() method in ./core/src/main/scala/org/apache/spark/util/Utils.scala You can find usage in doFetchFile(). FYI On Tue, Mar 17, 2015 at 6:52 AM, Hafiz Mujadid hafizmujadi...@gmail.com wrote: Hi experts! Is there any api in spark to download data from url? I

Re: 1.3 release

2015-03-17 Thread Sean Owen
OK, did you build with YARN support (-Pyarn)? and the right incantation of flags like -Phadoop-2.4 -Dhadoop.version=2.5.0-cdh5.3.2 or similar? On Tue, Mar 17, 2015 at 2:39 PM, Eric Friedman eric.d.fried...@gmail.com wrote: I did not find that the generic build worked. In fact I also haven't

Re: 1.3 release

2015-03-17 Thread Eric Friedman
I did not find that the generic build worked. In fact I also haven't gotten a build from source to work either, though that one might be a case of PEBCAK. In the former case I got errors about the build not having YARN support. On Sun, Mar 15, 2015 at 3:03 AM, Sean Owen so...@cloudera.com wrote:

org.apache.hadoop.hive.serde2.SerDeException: org.codehaus.jackson.JsonParseException

2015-03-17 Thread fanooos
I have a hadoop cluster and I need to query the data stored on the HDFS using spark sql thrift server. Spark sql thrift server is up and running. It is configured to read from HIVE table. The hive table is an external table that corresponding to set of files stored on HDFS. These files contains

Unable to saveAsParquetFile to HDFS since Spark 1.3.0

2015-03-17 Thread Franz Graf
Hi all, today we tested Spark 1.3.0. Everything went pretty fine except that I seem to be unable to save an RDD as parquet to HDFS. A minimum example is: import sqlContext.implicits._ // Reading works fine! val foo: RDD[String] = spark.textFile(hdfs://) // this works

Re: Can LBFGS be used on streaming data?

2015-03-17 Thread Jeremy Freeman
Hi Arunkumar, That looks like it should work. Logically, it’s similar to the implementation used by StreamingLinearRegression and StreamingLogisticRegression, see this class:

Re: Spark @ EC2: Futures timed out Ask timed out

2015-03-17 Thread Otis Gospodnetic
Hi Akhil, Thanks! I think that was it. Had to open a bunch of ports (didn't use spark-ec2, so it didn't do that for me) and the app works fine now. Otis -- Monitoring * Alerting * Anomaly Detection * Centralized Log Management Solr Elasticsearch Support * http://sematext.com/ On Tue, Mar

Why I didn't see the benefits of using KryoSerializer

2015-03-17 Thread java8964
Hi, I am new to Spark. I tried to understand the memory benefits of using KryoSerializer. I have this one box standalone test environment, which is 24 cores with 24G memory. I installed Hadoop 2.2 plus Spark 1.2.0. I put one text file in the hdfs about 1.2G. Here is the settings in the

Re: Spark SQL. Cast to Bigint

2015-03-17 Thread Masf
Hi Yin With HiveContext works well. Thanks!!! Regars. Miguel Angel. On Fri, Mar 13, 2015 at 3:18 PM, Yin Huai yh...@databricks.com wrote: Are you using SQLContext? Right now, the parser in the SQLContext is quite limited on the data type keywords that it handles (see here

High GC time

2015-03-17 Thread jatinpreet
Hi, I am getting very high GC time in my jobs. For smaller/real-time load, this becomes a real problem. Below are the details of a task I just ran. What could be the cause of such skewed GC times? 36 26010 SUCCESS PROCESS_LOCAL 2 / Slave1 2015/03/17 11:18:44 20 s

Re: Unable to saveAsParquetFile to HDFS since Spark 1.3.0

2015-03-17 Thread Cheng Lian
This has been fixed by https://github.com/apache/spark/pull/5020 On 3/18/15 12:24 AM, Franz Graf wrote: Hi all, today we tested Spark 1.3.0. Everything went pretty fine except that I seem to be unable to save an RDD as parquet to HDFS. A minimum example is: import sqlContext.implicits._ //

Spark yarn-client submission example?

2015-03-17 Thread Michal Klos
Hi, We have a Scala application and we want it to programmatically submit Spark jobs to a Spark-YARN cluster in yarn-client mode. We're running into a lot of classpath issues, e.g. once submitted it looks for jars in our parent Scala application's local directory, jars that it shouldn't need.

Re: HiveContext can't find registered function

2015-03-17 Thread Yin Huai
Seems basic_null_diluted_d was not resolved? Can you check if basic_null_diluted_d is in you table? On Tue, Mar 17, 2015 at 9:34 AM, Ophir Cohen oph...@gmail.com wrote: Hi Guys, I'm registering a function using: sqlc.registerFunction(makeEstEntry,ReutersDataFunctions.makeEstEntry _) Then I

  1   2   >