Re: [ANNOUNCE] Apache Spark 2.0.0-preview release

2016-05-26 Thread Sean Owen
I still don't see any artifacts in maven -- did it publish? http://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.spark%22 On Wed, May 25, 2016 at 10:30 AM, Reynold Xin wrote: > Yup I have published it to maven. Will post the link in a bit. > > One thing is that for

Creation of SparkML Estimators in Java broken?

2016-05-26 Thread Benjii519
Hello, Let me preface this with the fact that I am completely new to Spark and Scala, so I may be missing something basic. I have been looking at implementing a clustering algorithm on top of SparkML using Java, and ran into immediate problems. As a sanity check, I went to the Java API

Re: Dataset reduceByKey

2016-05-26 Thread Reynold Xin
Here's a ticket: https://issues.apache.org/jira/browse/SPARK-15598 On Fri, May 20, 2016 at 12:35 AM, Reynold Xin wrote: > Andres - this is great feedback. Let me think about it a little bit more > and reply later. > > > On Thu, May 19, 2016 at 11:12 AM, Andres Perez

Re: changed behavior for csv datasource and quoting in spark 2.0.0-SNAPSHOT

2016-05-26 Thread Reynold Xin
Yup - but the reason we did the null handling that way was for Python, which also affects Scala. On Thu, May 26, 2016 at 4:17 PM, Koert Kuipers wrote: > ok, thanks for creating ticket. > > just to be clear: my example was in scala > > On Thu, May 26, 2016 at 7:07 PM, Reynold

Re: changed behavior for csv datasource and quoting in spark 2.0.0-SNAPSHOT

2016-05-26 Thread Koert Kuipers
ok, thanks for creating ticket. just to be clear: my example was in scala On Thu, May 26, 2016 at 7:07 PM, Reynold Xin wrote: > This is unfortunately due to the way we set handle default values in > Python. I agree it doesn't follow the principle of least astonishment. > >

changed behavior for csv datasource and quoting in spark 2.0.0-SNAPSHOT

2016-05-26 Thread Koert Kuipers
in spark 1.6.1 we used: sqlContext.read .format("com.databricks.spark.csv") .delimiter("~") .option("quote", null) this effectively turned off quoting, which is a necessity for certain data formats where quoting is not supported and "\"" is a valid character itself in the data.

Re: JDBC Dialect for saving DataFrame into Vertica Table

2016-05-26 Thread Reynold Xin
It's probably a good idea to have the vertica dialect too, since it doesn't seem like it'd be too difficult to maintain. It is not going to be as performant as the native Vertica data source, but is going to be much lighter weight. On Thu, May 26, 2016 at 3:09 PM, Mohammed Guller

RE: JDBC Dialect for saving DataFrame into Vertica Table

2016-05-26 Thread Mohammed Guller
Vertica also provides a Spark connector. It was not GA the last time I looked at it, but available on the Vertica community site. Have you tried using the Vertica Spark connector instead of the JDBC driver? Mohammed Author: Big Data Analytics with

How to access the off-heap representation of cached data in Spark 2.0

2016-05-26 Thread jpivar...@gmail.com
Following up on an earlier thread , I would like to access the off-heap representation of cached data in Spark 2.0 in order to see how Spark might be linked to physics

Re: LiveListenerBus with started and stopped flags? Why both?

2016-05-26 Thread Shixiong(Ryan) Zhu
Just to prevent from restarting LiveListenerBus. The internal Thread cannot be restarted. On Wed, May 25, 2016 at 12:59 PM, Jacek Laskowski wrote: > Hi, > > I'm wondering why LiveListenerBus has two AtomicBoolean flags [1]? > Could it not have just one, say started? Why does

[RESULT][VOTE] Removing module maintainer process

2016-05-26 Thread Matei Zaharia
Thanks everyone for voting. With only +1 votes, the vote passes, so I'll update the contributor wiki appropriately. +1 votes: Matei Zaharia (binding) Mridul Muralidharan (binding) Andrew Or (binding) Sean Owen (binding) Nick Pentreath (binding) Tom Graves (binding) Imran Rashid (binding) Holden

Spark Job Execution halts during shuffle...

2016-05-26 Thread Priya Ch
Hello Team, I am trying to perform join 2 rdds where one is of size 800 MB and the other is 190 MB. During the join step, my job halts and I don't see progress in the execution. This is the message I see on console - INFO spark.MapOutputTrackerMasterEndPoint: Asked to send map output

Merging two datafiles

2016-05-26 Thread dvlpr
Hi everyone, I am doing some research in spark. I have one doubt: Can we merge or combine two datafiles and two indexfiles of different jobs (on same rdd) ? Please give me some ideas. Thank you! -- View this message in context:

Re: Labeling Jiras

2016-05-26 Thread Steve Loughran
> On 25 May 2016, at 23:08, Sean Owen wrote: > > Yeah I think using labels is fine -- just not if they're for someone's > internal purpose. I don't have a problem with using meaningful labels > if they're meaningful to everyone. In fact, I'd rather be using labels > rather

Re: [ANNOUNCE] Apache Spark 2.0.0-preview release

2016-05-26 Thread Gurvinder Singh
On 05/26/2016 02:38 AM, Matei Zaharia wrote: > Just wondering, what is the main use case for the Docker images -- to > develop apps locally or to deploy a cluster? I use docker images for both development and deploying on production cluster. As it makes sure I have the correct version of Java and