Re: Multi-Line JSON in SparkSQL

2015-05-05 Thread Joe Halliwell
@reynold, I’ll raise a JIRA today.@oliver, let’s discuss on the ticket? I suspect the algorithm is going to be bit fiddly and would definitely benefit from multiple heads. If possible, I think we should handle pathological cases like {“:”:”:”,{”{”:”}”}} correctly, rather than bailing out.

Re: Multi-Line JSON in SparkSQL

2015-05-05 Thread Ewan Higgs
FWIW, CSV has the same problem that renders it immune to naive partitioning. Consider the following RFC 4180 compliant record: 1,2," all,of,these,are,just,one,field ",4,5 Now, it's probably a terrible idea to give a file system awareness of actual file types, but couldn't HDFS handle this near

Re: Event generator for SPARK-Streaming from csv

2015-05-05 Thread anshu shukla
I know these methods , but i need to create events using the timestamps in the data tuples ,means every time a new tuple is generated using the timestamp in a CSV file .this will be useful to simulate the data rate with time just like real sensor data . On Fri, May 1, 2015 at 2:52 PM, Juan Rodrí

Re: Speeding up Spark build during development

2015-05-05 Thread Iulian Dragoș
I'm probably the only Eclipse user here, but it seems I have the best workflow :) At least for me things work as they should: once I imported projects in the workspace I can build and run/debug tests from the IDE. I only go to sbt when I need to re-create projects or I want to run the full test sui

Re: [discuss] ending support for Java 6?

2015-05-05 Thread Patrick Wendell
If there is broad consensus here to drop Java 1.6 in Spark 1.5, should we do an ANNOUNCE to user and dev? On Mon, May 4, 2015 at 7:24 PM, shane knapp wrote: > sgtm > > On Mon, May 4, 2015 at 11:23 AM, Patrick Wendell wrote: >> >> If we just set JAVA_HOME in dev/run-test-jenkins, I think it shoul

Pull request builder errors (taking Jenkins worker 3 offline)

2015-05-05 Thread Patrick Wendell
For unknown reasons, pull requests on Jenkins worker 3 have been failing with an exception[1]. After trying to fix this by clearing the ivy and maven caches on the node, I've given up and simply blacklisted that worker. [error] oro#oro;2.0.8!oro.jar origin location must be absolute: file:/home/jen

Re: Multi-Line JSON in SparkSQL

2015-05-05 Thread Joe Halliwell
I've raised the JSON-related ticket at https://issues.apache.org/jira/browse/SPARK-7366. @Ewan I think it would be great to support multiline CSV records too. The motivation is very similar but my instinct is that little/nothing of the implementation could be usefully shared, so it's better as a s

Spark/Mesos

2015-05-05 Thread Gidon Gershinsky
Hi all, I have a few questions on how Spark is integrated with Mesos - any details, or pointers to a design document / relevant source, will be much appreciated. I'm aware of this description, https://github.com/apache/spark/blob/master/docs/running-on-mesos.md But its pretty high-level as

Re: practical usage of the new "exactly-once" supporting DirectKafkaInputDStream

2015-05-05 Thread Mark Stewart
In case anyone else was having similar issues, the reordering and dropping of the reduceByKey solved the issues we were having. Thank you kindly, Mr. Koeninger. On Thu, Apr 30, 2015 at 3:06 PM, Cody Koeninger wrote: > In fact, you're using the 2 arg form of reduce by key to shrink it down to > 1

Re: practical usage of the new "exactly-once" supporting DirectKafkaInputDStream

2015-05-05 Thread Cody Koeninger
Glad that worked out for you. I updated the post on my github to hopefully clarify the issue. On Tue, May 5, 2015 at 9:36 AM, Mark Stewart wrote: > In case anyone else was having similar issues, the reordering and dropping > of the reduceByKey solved the issues we were having. Thank you kindly,

Scan Sharing in Spark

2015-05-05 Thread Quang-Nhat HOANG-XUAN
Hi everyone, I have two Spark jobs inside a Spark Application, which read from the same input file. They are executed in 2 threads. Right now, I cache the input file into memory before executing these two jobs. Are there another ways to share their same input with just only one read? I know ther

Re: Pull request builder errors (taking Jenkins worker 3 offline)

2015-05-05 Thread shane knapp
taking a look now. On Tue, May 5, 2015 at 3:23 AM, Patrick Wendell wrote: > For unknown reasons, pull requests on Jenkins worker 3 have been > failing with an exception[1]. After trying to fix this by clearing the > ivy and maven caches on the node, I've given up and simply blacklisted > that wo

Hive.get() called without HiveConf being already set on a yarn executor

2015-05-05 Thread Manku Timma
Looks like there is a case in TableReader.scala where Hive.get() is being called without already setting it via Hive.get(hiveconf). I am running in yarn-client mode (compiled with -Phive-provided and with hive-0.13.1a). Basically this means the broadcasted hiveconf is not getting used and the defau

Re: Pull request builder errors (taking Jenkins worker 3 offline)

2015-05-05 Thread shane knapp
hmm, still happening. looking deeper. On Tue, May 5, 2015 at 8:54 AM, shane knapp wrote: > taking a look now. > > On Tue, May 5, 2015 at 3:23 AM, Patrick Wendell > wrote: > >> For unknown reasons, pull requests on Jenkins worker 3 have been >> failing with an exception[1]. After trying to fix

Re: Scan Sharing in Spark

2015-05-05 Thread Evan R. Sparks
Scan sharing can indeed be a useful optimization in spark, because you amortize not only the time spent scanning over the data, but also time spent in task launch and scheduling overheads. Here's a trivial example in scala. I'm not aware of a place in SparkSQL where this is used - I'd imagine that

Re: Pull request builder errors (taking Jenkins worker 3 offline)

2015-05-05 Thread shane knapp
ok, i reset the maven cache on amp-jenkins-worker-03 and some stuff is currently building and not failing... i'll keep a close eye on this for now. On Tue, May 5, 2015 at 9:15 AM, shane knapp wrote: > hmm, still happening. looking deeper. > > On Tue, May 5, 2015 at 8:54 AM, shane knapp wrote:

Re: Spark/Mesos

2015-05-05 Thread Hector Yee
Speaking as a user of spark on mesos Yes it appears that each app appears as a separate framework on the mesos master In fine grained mode the number of executors goes up and down vs fixed in coarse. I would not run fine grained mode on a large cluster as it can potentially spin up a lot of execu

Re: Spark/Mesos

2015-05-05 Thread Timothy Chen
Hi Gidon, 1. Yes, each Spark application is wrapped in a new Mesos framework. 2. In fine grained mode, what happens is that Spark scheduler specifies a custom Mesos executor per slave, and each Mesos task is a Spark executor that will be launched by the Mesos executor. It's hard to determine what

Re: Typo on Spark SQL web page

2015-05-05 Thread Tony Stevenson
Kathy, Thank you. I have CCd the project so they can resolve this. On Tuesday, 5 May 2015, Kathy Wilson wrote: > There’s a typo in the title under Integrated on the Spark SQL web page ( > https://spark.apache.org/sql/): > > > > Seemlessly mix SQL queries with Spark programs. > > > > It shoul

New Kafka producer API

2015-05-05 Thread BenFradet
Hi, Since we're now supporting Kafka 0.8.2.1 , and that there is a new Producer API with this version, I was wondering if we should convert to this new API in KafkaTestUtils

Re: Thanking Test Partners

2015-05-05 Thread Imran Rashid
+1 testing is super important, it'll be good to give recognition for it. On Mon, May 4, 2015 at 5:46 PM, Patrick Wendell wrote: > Hey All, > > Community testing during the QA window is an important part of the > release cycle in Spark. It helps us deliver higher quality releases by > vetting ou

Re: [discuss] ending support for Java 6?

2015-05-05 Thread York, Brennon
+1 in favor of dropping Java1.6 support. +1 in favor of doing a wide ANNOUNCE to the user and dev groups declaring which version of Spark (sounds like 1.5) will drop support and when (if it isn¹t already posted somewhere) Spark 1.5 will release. On 5/5/15, 3:08 AM, "Patrick Wendell" wrote:

Re: New Kafka producer API

2015-05-05 Thread Cody Koeninger
Since that's an internal class used only for unit testing, what would the benefit be? On Tue, May 5, 2015 at 3:19 PM, BenFradet wrote: > Hi, > > Since we're now supporting Kafka 0.8.2.1 > , and that there is a new > Producer API

Re: New Kafka producer API

2015-05-05 Thread BenFradet
Even if it's only used for testing and the examples, why not move ahead of the deprecation and gain some performance along the way. Plus, regarding the examples, I think it's good practice to use the recommended API and not the legacy one. -- View this message in context: http://apache-spark-d

NP-Complete Design Choices in Spark Implementation

2015-05-05 Thread Junior
Is there any NP-Complete related problem that Spark or Spark Streaming engineers needed to address efficiently during design/code implementation in order to achieve satisfactory performance? Can you mention some example of this NP-Complete and the implementation choice? -- View this message in c

Re: New Kafka producer API

2015-05-05 Thread Cody Koeninger
Regarding performance, keep in mind we'd probably have to turn all those async calls into blocking calls for the unit tests On Tue, May 5, 2015 at 3:44 PM, BenFradet wrote: > Even if it's only used for testing and the examples, why not move ahead of > the deprecation and gain some performance al

Re: Scan Sharing in Spark

2015-05-05 Thread Quang-Nhat HOANG-XUAN
Hi, Beside caching, is it possible if an RDD has multiple child RDDs? So I can read the input one and produce multiple outputs for multiple jobs which share the input. On May 5, 2015 6:24 PM, "Evan R. Sparks" wrote: > Scan sharing can indeed be a useful optimization in spark, because you > amort

Re: New Kafka producer API

2015-05-05 Thread BenFradet
Yes that might be true, I will have to test that. -- View this message in context: http://apache-spark-developers-list.1001551.n3.nabble.com/New-Kafka-producer-API-tp12050p12058.html Sent from the Apache Spark Developers List mailing list archive at Nabble.com.

Re: [discuss] ending support for Java 6?

2015-05-05 Thread Tom Graves
+1. I haven't seen major objections here so I would say send announcement and see if any users have objections Tom On Tuesday, May 5, 2015 5:09 AM, Patrick Wendell wrote: If there is broad consensus here to drop Java 1.6 in Spark 1.5, should we do an ANNOUNCE to user and dev? On M

Re: [discuss] ending support for Java 6?

2015-05-05 Thread shane knapp
+1 to an announce to user and dev. java6 is so old and sad. On Tue, May 5, 2015 at 2:24 PM, Tom Graves wrote: > +1. I haven't seen major objections here so I would say send announcement > and see if any users have objections > > Tom > > > > On Tuesday, May 5, 2015 5:09 AM, Patrick Wendell >

Re: Pull request builder errors (taking Jenkins worker 3 offline)

2015-05-05 Thread shane knapp
alright, this is happening again w/this worker and i will be taking it offline for further investigation. i'm OOO for the rest of the day, but will check in again later this evening. On Tue, May 5, 2015 at 9:33 AM, shane knapp wrote: > ok, i reset the maven cache on amp-jenkins-worker-03 and so

Re: [discuss] ending support for Java 6?

2015-05-05 Thread Reynold Xin
OK I sent an email. On Tue, May 5, 2015 at 2:47 PM, shane knapp wrote: > +1 to an announce to user and dev. java6 is so old and sad. > > On Tue, May 5, 2015 at 2:24 PM, Tom Graves wrote: > >> +1. I haven't seen major objections here so I would say send announcement >> and see if any users hav

[ANNOUNCE] Ending Java 6 support in Spark 1.5 (Sep 2015)

2015-05-05 Thread Reynold Xin
Hi all, We will drop support for Java 6 starting Spark 1.5, tentative scheduled to be released in Sep 2015. Spark 1.4, scheduled to be released in June 2015, will be the last minor release that supports Java 6. That is to say: Spark 1.4.x (~ Jun 2015): will work with Java 6, 7, 8. Spark 1.5+ (~

Re: LDA and PageRank Using GraphX

2015-05-05 Thread 吴明瑜
There is a PageRank algorithm in the lib package of graphx. And you can find an example to invoke it in SynthBenchmark.scala in org.apache.spark.examples.graphx. 2015-05-03 16:52 GMT+08:00 Praveen Kumar Muthuswamy : > Hi All, > I am looking to run LDA for topic modeling and page rank algorithms

Re: [ANNOUNCE] Ending Java 6 support in Spark 1.5 (Sep 2015)

2015-05-05 Thread Sean Owen
OK to file a JIRA to scrape out a few Java 6-specific things in the code? and/or close issues about working with Java 6 if they're not going to be resolved for 1.4? I suppose this means the master builds and PR builder in Jenkins should simply continue to use Java 7 then. On Tue, May 5, 2015 at 1

Re: [ANNOUNCE] Ending Java 6 support in Spark 1.5 (Sep 2015)

2015-05-05 Thread Reynold Xin
Sean - Please do. On Tue, May 5, 2015 at 10:57 PM, Sean Owen wrote: > OK to file a JIRA to scrape out a few Java 6-specific things in the > code? and/or close issues about working with Java 6 if they're not > going to be resolved for 1.4? > > I suppose this means the master builds and PR builder

Re: Speeding up Spark build during development

2015-05-05 Thread Pramod Biligiri
I had to make a small change to Emre's suggestion above, in order for my changes to get picked up. This worked for me: mvn --projects sql/core -DskipTests install #not package mvn --projects assembly/ -DskipTests install Pramod On Tue, May 5, 2015 at 2:36 AM, Iulian Dragoș wrote: > I'm probably

Re: Pickling error when attempting to add a method in pyspark

2015-05-05 Thread Xiangrui Meng
Hi Stephen, I think it would be easier to see what you implemented by showing the branch diff link on github. There are couple utility class to make Rating work between Scala and Python: 1. serializer: https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/mllib/api/py

Re: OOM error with GMMs on 4GB dataset

2015-05-05 Thread Xiangrui Meng
Did you set `--driver-memory` with spark-submit? -Xiangrui On Mon, May 4, 2015 at 5:16 PM, Vinay Muttineni wrote: > Hi, I am training a GMM with 10 gaussians on a 4 GB dataset(720,000 * 760). > The spark (1.3.1) job is allocated 120 executors with 6GB each and the > driver also has 6GB. > Spark C

Re: Typo on Spark SQL web page

2015-05-05 Thread Sean Owen
Oops, fixed now. On Tue, May 5, 2015 at 9:09 PM, Tony Stevenson wrote: > Kathy, > > Thank you. I have CCd the project so they can resolve this. > > > > On Tuesday, 5 May 2015, Kathy Wilson wrote: > >> There’s a typo in the title under Integrated on the Spark SQL web page ( >> https://spark.apac

Re: [discuss] ending support for Java 6?

2015-05-05 Thread Xiangrui Meng
+1. One issue with dropping Java 6: if we use Java 7 to build the assembly jar, it will use zip64. Could Python 2.x (or even 3.x) be able to load zip64 files on PYTHONPATH? -Xiangrui On Tue, May 5, 2015 at 3:25 PM, Reynold Xin wrote: > OK I sent an email. > > > On Tue, May 5, 2015 at 2:47 PM, sha

Re: [discuss] ending support for Java 6?

2015-05-05 Thread Reynold Xin
@tgraves can chime in, but I think this pr aims to fix it: https://github.com/apache/spark/pull/5580 We should probably get that in for 1.4. On Tue, May 5, 2015 at 11:46 PM, Xiangrui Meng wrote: > +1. One issue with dropping Java 6: if we use Java 7 to build the > assembly jar, it will use zip