I'm working on SPARK-6235 Address various 2G limits.

2016-09-08 Thread liguoqiang (I)
Hi, all I'm working on SPARK-6235 Address various 2G limits, welcome to ask questions or make comments. The jira: https://issues.apache.org/jira/browse/SPARK-6235 and the code: https://github.com/apache/spark/pull/14995 Thanks Guoqiang Li

Re: @scala.annotation.varargs or @_root_.scala.annotation.varargs?

2016-09-08 Thread Reynold Xin
Yea but the earlier email was asking they were introduced in the first place. On Friday, September 9, 2016, Marcelo Vanzin wrote: > Not after SPARK-14642, right? > > On Thu, Sep 8, 2016 at 5:07 PM, Reynold Xin > wrote: > > There is a

Re: @scala.annotation.varargs or @_root_.scala.annotation.varargs?

2016-09-08 Thread Marcelo Vanzin
Not after SPARK-14642, right? On Thu, Sep 8, 2016 at 5:07 PM, Reynold Xin wrote: > There is a package called scala. > > > On Friday, September 9, 2016, Hyukjin Kwon wrote: >> >> I was also actually wondering why it is being written like this. >> >> I

Re: @scala.annotation.varargs or @_root_.scala.annotation.varargs?

2016-09-08 Thread Reynold Xin
There is a package called scala. On Friday, September 9, 2016, Hyukjin Kwon wrote: > I was also actually wondering why it is being written like this. > > I actually took a look for this before and wanted to fix them but I found >

Re: @scala.annotation.varargs or @_root_.scala.annotation.varargs?

2016-09-08 Thread Hyukjin Kwon
I was also actually wondering why it is being written like this. I actually took a look for this before and wanted to fix them but I found https://github.com/apache/spark/pull/12077/files#r58041468 So, I kind of persuaded myself that committers already know about it and there is a reason for

Re: @scala.annotation.varargs or @_root_.scala.annotation.varargs?

2016-09-08 Thread Jakob Odersky
+1 to Sean's answer, importing varargs. In this case the _root_ is also unnecessary (it would be required in case you were using it in a nested package called "scala" itself) On Thu, Sep 8, 2016 at 9:27 AM, Sean Owen wrote: > I think the @_root_ version is redundant because >

Re: FileStreamSource source checks path eagerly?

2016-09-08 Thread Matei Zaharia
This source is meant to be used for a shared file system such as HDFS or NFS, where both the driver and the workers can see the same folders. There's no support in Spark for just working with local files on different workers. Matei > On Sep 8, 2016, at 2:23 AM, Jacek Laskowski

Re: @scala.annotation.varargs or @_root_.scala.annotation.varargs?

2016-09-08 Thread Sean Owen
I think the @_root_ version is redundant because @scala.annotation.varargs is redundant. Actually wouldn't we just import varargs and write @varargs? On Thu, Sep 8, 2016 at 1:24 PM, Jacek Laskowski wrote: > Hi, > > The code is not consistent with @scala.annotation.varargs

@scala.annotation.varargs or @_root_.scala.annotation.varargs?

2016-09-08 Thread Jacek Laskowski
Hi, The code is not consistent with @scala.annotation.varargs annotation. There are classes with @scala.annotation.varargs like DataFrameReader or functions as well as examples of @_root_.scala.annotation.varargs, e.g. Window or UserDefinedAggregateFunction. I think it should be consistent and

Re: FileStreamSource source checks path eagerly?

2016-09-08 Thread Jacek Laskowski
Hi Steve, Thank you for more source-oriented answer. Helped but didn't explain the reason for such eagerness. The file(s) might not be on the driver but on executors only where the Spark job(s) run. I don't see why Spark should check the file(s) regardless of glob pattern being used. You see my

Re: FileStreamSource source checks path eagerly?

2016-09-08 Thread Steve Loughran
failfast generally means that you find problems sooner rather than later, and here, potentially, that your code runs but simply returns empty data without any obvious cue as to what is wrong. As is always good in OSS, follow those stack trace links to see what they say: // Check

Re: FileStreamSource source checks path eagerly?

2016-09-08 Thread Jacek Laskowski
On Thu, Sep 8, 2016 at 9:03 AM, Fred Reiss wrote: > I suppose the type-inference-time check for the presence of the input > directory could be moved to the FileStreamSource's initialization. But if > the directory isn't there when the source is being created, it probably >

FileStreamSource source checks path eagerly?

2016-09-08 Thread Jacek Laskowski
Hi, I'm wondering what's the rationale for checking the path option eagerly in FileStreamSource? My thinking is that until start is called there's no processing going on that is supposed to happen on executors (not the driver) with the path available. I could (and perhaps should) use dfs but