Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-12 Thread Nan Zhu
ah, I see,   

I think it’s hard to do something like fs.delete() in spark code (it’s scary as 
we discussed in the previous PR )

so if you want (C), I guess you have to do some delete work manually  

Best,  

--  
Nan Zhu


On Thursday, June 12, 2014 at 3:31 PM, Daniel Siegmann wrote:

> I do not want the behavior of (A) - that is dangerous and should only be 
> enabled to account for legacy code. Personally, I think this option should 
> eventually be removed.
>  
> I want the option (C), to have Spark delete any existing part files before 
> creating any new output. I don't necessarily want this to be a global option, 
> but one on the API for saveTextFile (i.e. an additional boolean parameter).
>  
> As it stands now, I need to precede every saveTextFile call with my own 
> deletion code.
>  
> In other words, instead of writing ...
>  
> if ( cleanOutput ) { MyUtil.clean(outputDir) }
> rdd.writeTextFile( outputDir )
>  
> I'd like to write
>  
> rdd.writeTextFile(outputDir, cleanOutput)
>  
> Does that make sense?
>  
>  
>  
>  
> On Thu, Jun 12, 2014 at 2:51 PM, Nan Zhu  (mailto:zhunanmcg...@gmail.com)> wrote:
> > Actually this has been merged to the master branch  
> >  
> > https://github.com/apache/spark/pull/947  
> >  
> > --  
> > Nan Zhu
> >  
> >  
> > On Thursday, June 12, 2014 at 2:39 PM, Daniel Siegmann wrote:
> >  
> > > The old behavior (A) was dangerous, so it's good that (B) is now the 
> > > default. But in some cases I really do want to replace the old data, as 
> > > per (C). For example, I may rerun a previous computation (perhaps the 
> > > input data was corrupt and I'm rerunning with good input).
> > >  
> > > Currently I have to write separate code to remove the files before 
> > > calling Spark. It would be very convenient if Spark could do this for me. 
> > > Has anyone created a JIRA issue to support (C)?
> > >  
> > >  
> > > On Mon, Jun 9, 2014 at 3:02 AM, Aaron Davidson  > > (mailto:ilike...@gmail.com)> wrote:
> > > > It is not a very good idea to save the results in the exact same place 
> > > > as the data. Any failures during the job could lead to corrupted data, 
> > > > because recomputing the lost partitions would involve reading the 
> > > > original (now-nonexistent) data.
> > > >  
> > > > As such, the only "safe" way to do this would be to do as you said, and 
> > > > only delete the input data once the entire output has been successfully 
> > > > created.
> > > >  
> > > >  
> > > > On Sun, Jun 8, 2014 at 10:32 PM, innowireless TaeYun Kim 
> > > > mailto:taeyun@innowireless.co.kr)> 
> > > > wrote:
> > > > > Without (C), what is the best practice to implement the following 
> > > > > scenario?
> > > > >  
> > > > > 1. rdd = sc.textFile(FileA)
> > > > > 2. rdd = rdd.map(...)  // actually modifying the rdd
> > > > > 3. rdd.saveAsTextFile(FileA)
> > > > >  
> > > > > Since the rdd transformation is 'lazy', rdd will not materialize until
> > > > > saveAsTextFile(), so FileA must still exist, but it must be deleted 
> > > > > before
> > > > > saveAsTextFile().
> > > > >  
> > > > > What I can think is:
> > > > >  
> > > > > 3. rdd.saveAsTextFile(TempFile)
> > > > > 4. delete FileA
> > > > > 5. rename TempFile to FileA
> > > > >  
> > > > > This is not very convenient...
> > > > >  
> > > > > Thanks.
> > > > >  
> > > > > -Original Message-
> > > > > From: Patrick Wendell [mailto:pwend...@gmail.com]
> > > > > Sent: Tuesday, June 03, 2014 11:40 AM
> > > > > To: user@spark.apache.org (mailto:user@spark.apache.org)
> > > > > Subject: Re: How can I make Spark 1.0 saveAsTextFile to overwrite 
> > > > > existing
> > > > > file
> > > > >  
> > > > > (A) Semantics in Spark 0.9 and earlier: Spark will ignore Hadoo's 
> > > > > output
> > > > > format check and overwrite files in the destination directory.
> > > > > But it won't clobber the directory entirely. I.e. if the directory 
> > > > > already
> > > > > had "part1" "part2" "part3" "part4&

Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-12 Thread Daniel Siegmann
I do not want the behavior of (A) - that is dangerous and should only be
enabled to account for legacy code. Personally, I think this option should
eventually be removed.

I want the option (C), to have Spark delete any existing part files before
creating any new output. I don't necessarily want this to be a global
option, but one on the API for saveTextFile (i.e. an additional boolean
parameter).

As it stands now, I need to precede every saveTextFile call with my own
deletion code.

In other words, instead of writing ...

if ( cleanOutput ) { MyUtil.clean(outputDir) }
rdd.writeTextFile( outputDir )

I'd like to write

rdd.writeTextFile(outputDir, cleanOutput)

Does that make sense?




On Thu, Jun 12, 2014 at 2:51 PM, Nan Zhu  wrote:

> Actually this has been merged to the master branch
>
> https://github.com/apache/spark/pull/947
>
> --
> Nan Zhu
>
> On Thursday, June 12, 2014 at 2:39 PM, Daniel Siegmann wrote:
>
> The old behavior (A) was dangerous, so it's good that (B) is now the
> default. But in some cases I really do want to replace the old data, as per
> (C). For example, I may rerun a previous computation (perhaps the input
> data was corrupt and I'm rerunning with good input).
>
> Currently I have to write separate code to remove the files before calling
> Spark. It would be very convenient if Spark could do this for me. Has
> anyone created a JIRA issue to support (C)?
>
>
> On Mon, Jun 9, 2014 at 3:02 AM, Aaron Davidson  wrote:
>
> It is not a very good idea to save the results in the exact same place as
> the data. Any failures during the job could lead to corrupted data, because
> recomputing the lost partitions would involve reading the original
> (now-nonexistent) data.
>
> As such, the only "safe" way to do this would be to do as you said, and
> only delete the input data once the entire output has been successfully
> created.
>
>
> On Sun, Jun 8, 2014 at 10:32 PM, innowireless TaeYun Kim <
> taeyun@innowireless.co.kr> wrote:
>
> Without (C), what is the best practice to implement the following scenario?
>
> 1. rdd = sc.textFile(FileA)
> 2. rdd = rdd.map(...)  // actually modifying the rdd
> 3. rdd.saveAsTextFile(FileA)
>
> Since the rdd transformation is 'lazy', rdd will not materialize until
> saveAsTextFile(), so FileA must still exist, but it must be deleted before
> saveAsTextFile().
>
> What I can think is:
>
> 3. rdd.saveAsTextFile(TempFile)
> 4. delete FileA
> 5. rename TempFile to FileA
>
> This is not very convenient...
>
> Thanks.
>
> -----Original Message-
> From: Patrick Wendell [mailto:pwend...@gmail.com]
> Sent: Tuesday, June 03, 2014 11:40 AM
> To: user@spark.apache.org
> Subject: Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing
> file
>
> (A) Semantics in Spark 0.9 and earlier: Spark will ignore Hadoo's output
> format check and overwrite files in the destination directory.
> But it won't clobber the directory entirely. I.e. if the directory already
> had "part1" "part2" "part3" "part4" and you write a new job outputing only
> two files ("part1", "part2") then it would leave the other two files
> intact,
> confusingly.
>
> (B) Semantics in Spark 1.0 and earlier: Runs Hadoop OutputFormat check
> which
> means the directory must not exist already or an excpetion is thrown.
>
> (C) Semantics proposed by Nicholas Chammas in this thread (AFAIK):
> Spark will delete/clobber an existing destination directory if it exists,
> then fully over-write it with new data.
>
> I'm fine to add a flag that allows (B) for backwards-compatibility reasons,
> but my point was I'd prefer not to have (C) even though I see some cases
> where it would be useful.
>
> - Patrick
>
> On Mon, Jun 2, 2014 at 4:25 PM, Sean Owen  wrote:
> > Is there a third way? Unless I miss something. Hadoop's OutputFormat
> > wants the target dir to not exist no matter what, so it's just a
> > question of whether Spark deletes it for you or errors.
> >
> > On Tue, Jun 3, 2014 at 12:22 AM, Patrick Wendell 
> wrote:
> >> We can just add back a flag to make it backwards compatible - it was
> >> just missed during the original PR.
> >>
> >> Adding a *third* set of "clobber" semantics, I'm slightly -1 on that
> >> for the following reasons:
> >>
> >> 1. It's scary to have Spark recursively deleting user files, could
> >> easily lead to users deleting data by mistake if they don't
> >> understand the exact semantics.
> >> 2. It would introduce a third set of seman

Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-12 Thread Nan Zhu
Actually this has been merged to the master branch 

https://github.com/apache/spark/pull/947 

-- 
Nan Zhu


On Thursday, June 12, 2014 at 2:39 PM, Daniel Siegmann wrote:

> The old behavior (A) was dangerous, so it's good that (B) is now the default. 
> But in some cases I really do want to replace the old data, as per (C). For 
> example, I may rerun a previous computation (perhaps the input data was 
> corrupt and I'm rerunning with good input).
> 
> Currently I have to write separate code to remove the files before calling 
> Spark. It would be very convenient if Spark could do this for me. Has anyone 
> created a JIRA issue to support (C)?
> 
> 
> On Mon, Jun 9, 2014 at 3:02 AM, Aaron Davidson  (mailto:ilike...@gmail.com)> wrote:
> > It is not a very good idea to save the results in the exact same place as 
> > the data. Any failures during the job could lead to corrupted data, because 
> > recomputing the lost partitions would involve reading the original 
> > (now-nonexistent) data.
> > 
> > As such, the only "safe" way to do this would be to do as you said, and 
> > only delete the input data once the entire output has been successfully 
> > created.
> > 
> > 
> > On Sun, Jun 8, 2014 at 10:32 PM, innowireless TaeYun Kim 
> > mailto:taeyun@innowireless.co.kr)> 
> > wrote:
> > > Without (C), what is the best practice to implement the following 
> > > scenario?
> > > 
> > > 1. rdd = sc.textFile(FileA)
> > > 2. rdd = rdd.map(...)  // actually modifying the rdd
> > > 3. rdd.saveAsTextFile(FileA)
> > > 
> > > Since the rdd transformation is 'lazy', rdd will not materialize until
> > > saveAsTextFile(), so FileA must still exist, but it must be deleted before
> > > saveAsTextFile().
> > > 
> > > What I can think is:
> > > 
> > > 3. rdd.saveAsTextFile(TempFile)
> > > 4. delete FileA
> > > 5. rename TempFile to FileA
> > > 
> > > This is not very convenient...
> > > 
> > > Thanks.
> > > 
> > > -Original Message-
> > > From: Patrick Wendell [mailto:pwend...@gmail.com]
> > > Sent: Tuesday, June 03, 2014 11:40 AM
> > > To: user@spark.apache.org (mailto:user@spark.apache.org)
> > > Subject: Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing
> > > file
> > > 
> > > (A) Semantics in Spark 0.9 and earlier: Spark will ignore Hadoo's output
> > > format check and overwrite files in the destination directory.
> > > But it won't clobber the directory entirely. I.e. if the directory already
> > > had "part1" "part2" "part3" "part4" and you write a new job outputing only
> > > two files ("part1", "part2") then it would leave the other two files 
> > > intact,
> > > confusingly.
> > > 
> > > (B) Semantics in Spark 1.0 and earlier: Runs Hadoop OutputFormat check 
> > > which
> > > means the directory must not exist already or an excpetion is thrown.
> > > 
> > > (C) Semantics proposed by Nicholas Chammas in this thread (AFAIK):
> > > Spark will delete/clobber an existing destination directory if it exists,
> > > then fully over-write it with new data.
> > > 
> > > I'm fine to add a flag that allows (B) for backwards-compatibility 
> > > reasons,
> > > but my point was I'd prefer not to have (C) even though I see some cases
> > > where it would be useful.
> > > 
> > > - Patrick
> > > 
> > > On Mon, Jun 2, 2014 at 4:25 PM, Sean Owen  > > (mailto:so...@cloudera.com)> wrote:
> > > > Is there a third way? Unless I miss something. Hadoop's OutputFormat
> > > > wants the target dir to not exist no matter what, so it's just a
> > > > question of whether Spark deletes it for you or errors.
> > > >
> > > > On Tue, Jun 3, 2014 at 12:22 AM, Patrick Wendell  > > > (mailto:pwend...@gmail.com)>
> > > wrote:
> > > >> We can just add back a flag to make it backwards compatible - it was
> > > >> just missed during the original PR.
> > > >>
> > > >> Adding a *third* set of "clobber" semantics, I'm slightly -1 on that
> > > >> for the following reasons:
> > > >>
> > > >> 1. It's scary to have Spark recursively deleting user files, could
> > > >> easily lead to users deleting data by mistake if they don't
> > > >> understand the exact semantics.
> > > >> 2. It would introduce a third set of semantics here for saveAsXX...
> > > >> 3. It's trivial for users to implement this with two lines of code
> > > >> (if output dir exists, delete it) before calling saveAsHadoopFile.
> > > >>
> > > >> - Patrick
> > > >>
> > > 
> > 
> 
> 
> 
> -- 
> Daniel Siegmann, Software Developer
> Velos
> Accelerating Machine Learning
> 
> 440 NINTH AVENUE, 11TH FLOOR, NEW YORK, NY 10001
> E: daniel.siegm...@velos.io (mailto:daniel.siegm...@velos.io) W: www.velos.io 
> (http://www.velos.io) 



Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-12 Thread Daniel Siegmann
The old behavior (A) was dangerous, so it's good that (B) is now the
default. But in some cases I really do want to replace the old data, as per
(C). For example, I may rerun a previous computation (perhaps the input
data was corrupt and I'm rerunning with good input).

Currently I have to write separate code to remove the files before calling
Spark. It would be very convenient if Spark could do this for me. Has
anyone created a JIRA issue to support (C)?


On Mon, Jun 9, 2014 at 3:02 AM, Aaron Davidson  wrote:

> It is not a very good idea to save the results in the exact same place as
> the data. Any failures during the job could lead to corrupted data, because
> recomputing the lost partitions would involve reading the original
> (now-nonexistent) data.
>
> As such, the only "safe" way to do this would be to do as you said, and
> only delete the input data once the entire output has been successfully
> created.
>
>
> On Sun, Jun 8, 2014 at 10:32 PM, innowireless TaeYun Kim <
> taeyun@innowireless.co.kr> wrote:
>
>> Without (C), what is the best practice to implement the following
>> scenario?
>>
>> 1. rdd = sc.textFile(FileA)
>> 2. rdd = rdd.map(...)  // actually modifying the rdd
>> 3. rdd.saveAsTextFile(FileA)
>>
>> Since the rdd transformation is 'lazy', rdd will not materialize until
>> saveAsTextFile(), so FileA must still exist, but it must be deleted before
>> saveAsTextFile().
>>
>> What I can think is:
>>
>> 3. rdd.saveAsTextFile(TempFile)
>> 4. delete FileA
>> 5. rename TempFile to FileA
>>
>> This is not very convenient...
>>
>> Thanks.
>>
>> -----Original Message-----
>> From: Patrick Wendell [mailto:pwend...@gmail.com]
>> Sent: Tuesday, June 03, 2014 11:40 AM
>> To: user@spark.apache.org
>> Subject: Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing
>> file
>>
>> (A) Semantics in Spark 0.9 and earlier: Spark will ignore Hadoo's output
>> format check and overwrite files in the destination directory.
>> But it won't clobber the directory entirely. I.e. if the directory already
>> had "part1" "part2" "part3" "part4" and you write a new job outputing only
>> two files ("part1", "part2") then it would leave the other two files
>> intact,
>> confusingly.
>>
>> (B) Semantics in Spark 1.0 and earlier: Runs Hadoop OutputFormat check
>> which
>> means the directory must not exist already or an excpetion is thrown.
>>
>> (C) Semantics proposed by Nicholas Chammas in this thread (AFAIK):
>> Spark will delete/clobber an existing destination directory if it exists,
>> then fully over-write it with new data.
>>
>> I'm fine to add a flag that allows (B) for backwards-compatibility
>> reasons,
>> but my point was I'd prefer not to have (C) even though I see some cases
>> where it would be useful.
>>
>> - Patrick
>>
>> On Mon, Jun 2, 2014 at 4:25 PM, Sean Owen  wrote:
>> > Is there a third way? Unless I miss something. Hadoop's OutputFormat
>> > wants the target dir to not exist no matter what, so it's just a
>> > question of whether Spark deletes it for you or errors.
>> >
>> > On Tue, Jun 3, 2014 at 12:22 AM, Patrick Wendell 
>> wrote:
>> >> We can just add back a flag to make it backwards compatible - it was
>> >> just missed during the original PR.
>> >>
>> >> Adding a *third* set of "clobber" semantics, I'm slightly -1 on that
>> >> for the following reasons:
>> >>
>> >> 1. It's scary to have Spark recursively deleting user files, could
>> >> easily lead to users deleting data by mistake if they don't
>> >> understand the exact semantics.
>> >> 2. It would introduce a third set of semantics here for saveAsXX...
>> >> 3. It's trivial for users to implement this with two lines of code
>> >> (if output dir exists, delete it) before calling saveAsHadoopFile.
>> >>
>> >> - Patrick
>> >>
>>
>>
>


-- 
Daniel Siegmann, Software Developer
Velos
Accelerating Machine Learning

440 NINTH AVENUE, 11TH FLOOR, NEW YORK, NY 10001
E: daniel.siegm...@velos.io W: www.velos.io


Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-09 Thread Aaron Davidson
It is not a very good idea to save the results in the exact same place as
the data. Any failures during the job could lead to corrupted data, because
recomputing the lost partitions would involve reading the original
(now-nonexistent) data.

As such, the only "safe" way to do this would be to do as you said, and
only delete the input data once the entire output has been successfully
created.


On Sun, Jun 8, 2014 at 10:32 PM, innowireless TaeYun Kim <
taeyun@innowireless.co.kr> wrote:

> Without (C), what is the best practice to implement the following scenario?
>
> 1. rdd = sc.textFile(FileA)
> 2. rdd = rdd.map(...)  // actually modifying the rdd
> 3. rdd.saveAsTextFile(FileA)
>
> Since the rdd transformation is 'lazy', rdd will not materialize until
> saveAsTextFile(), so FileA must still exist, but it must be deleted before
> saveAsTextFile().
>
> What I can think is:
>
> 3. rdd.saveAsTextFile(TempFile)
> 4. delete FileA
> 5. rename TempFile to FileA
>
> This is not very convenient...
>
> Thanks.
>
> -Original Message-
> From: Patrick Wendell [mailto:pwend...@gmail.com]
> Sent: Tuesday, June 03, 2014 11:40 AM
> To: user@spark.apache.org
> Subject: Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing
> file
>
> (A) Semantics in Spark 0.9 and earlier: Spark will ignore Hadoo's output
> format check and overwrite files in the destination directory.
> But it won't clobber the directory entirely. I.e. if the directory already
> had "part1" "part2" "part3" "part4" and you write a new job outputing only
> two files ("part1", "part2") then it would leave the other two files
> intact,
> confusingly.
>
> (B) Semantics in Spark 1.0 and earlier: Runs Hadoop OutputFormat check
> which
> means the directory must not exist already or an excpetion is thrown.
>
> (C) Semantics proposed by Nicholas Chammas in this thread (AFAIK):
> Spark will delete/clobber an existing destination directory if it exists,
> then fully over-write it with new data.
>
> I'm fine to add a flag that allows (B) for backwards-compatibility reasons,
> but my point was I'd prefer not to have (C) even though I see some cases
> where it would be useful.
>
> - Patrick
>
> On Mon, Jun 2, 2014 at 4:25 PM, Sean Owen  wrote:
> > Is there a third way? Unless I miss something. Hadoop's OutputFormat
> > wants the target dir to not exist no matter what, so it's just a
> > question of whether Spark deletes it for you or errors.
> >
> > On Tue, Jun 3, 2014 at 12:22 AM, Patrick Wendell 
> wrote:
> >> We can just add back a flag to make it backwards compatible - it was
> >> just missed during the original PR.
> >>
> >> Adding a *third* set of "clobber" semantics, I'm slightly -1 on that
> >> for the following reasons:
> >>
> >> 1. It's scary to have Spark recursively deleting user files, could
> >> easily lead to users deleting data by mistake if they don't
> >> understand the exact semantics.
> >> 2. It would introduce a third set of semantics here for saveAsXX...
> >> 3. It's trivial for users to implement this with two lines of code
> >> (if output dir exists, delete it) before calling saveAsHadoopFile.
> >>
> >> - Patrick
> >>
>
>


RE: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-08 Thread innowireless TaeYun Kim
Without (C), what is the best practice to implement the following scenario?

1. rdd = sc.textFile(FileA)
2. rdd = rdd.map(...)  // actually modifying the rdd
3. rdd.saveAsTextFile(FileA)

Since the rdd transformation is 'lazy', rdd will not materialize until
saveAsTextFile(), so FileA must still exist, but it must be deleted before
saveAsTextFile().

What I can think is:

3. rdd.saveAsTextFile(TempFile)
4. delete FileA
5. rename TempFile to FileA

This is not very convenient...

Thanks.

-Original Message-
From: Patrick Wendell [mailto:pwend...@gmail.com] 
Sent: Tuesday, June 03, 2014 11:40 AM
To: user@spark.apache.org
Subject: Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing
file

(A) Semantics in Spark 0.9 and earlier: Spark will ignore Hadoo's output
format check and overwrite files in the destination directory.
But it won't clobber the directory entirely. I.e. if the directory already
had "part1" "part2" "part3" "part4" and you write a new job outputing only
two files ("part1", "part2") then it would leave the other two files intact,
confusingly.

(B) Semantics in Spark 1.0 and earlier: Runs Hadoop OutputFormat check which
means the directory must not exist already or an excpetion is thrown.

(C) Semantics proposed by Nicholas Chammas in this thread (AFAIK):
Spark will delete/clobber an existing destination directory if it exists,
then fully over-write it with new data.

I'm fine to add a flag that allows (B) for backwards-compatibility reasons,
but my point was I'd prefer not to have (C) even though I see some cases
where it would be useful.

- Patrick

On Mon, Jun 2, 2014 at 4:25 PM, Sean Owen  wrote:
> Is there a third way? Unless I miss something. Hadoop's OutputFormat 
> wants the target dir to not exist no matter what, so it's just a 
> question of whether Spark deletes it for you or errors.
>
> On Tue, Jun 3, 2014 at 12:22 AM, Patrick Wendell 
wrote:
>> We can just add back a flag to make it backwards compatible - it was 
>> just missed during the original PR.
>>
>> Adding a *third* set of "clobber" semantics, I'm slightly -1 on that 
>> for the following reasons:
>>
>> 1. It's scary to have Spark recursively deleting user files, could 
>> easily lead to users deleting data by mistake if they don't 
>> understand the exact semantics.
>> 2. It would introduce a third set of semantics here for saveAsXX...
>> 3. It's trivial for users to implement this with two lines of code 
>> (if output dir exists, delete it) before calling saveAsHadoopFile.
>>
>> - Patrick
>>



Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-03 Thread Sean Owen
Ah, the output directory check was just not executed in the past. I
thought it deleted the files. A third way indeed.

FWIW I also think (B) is best. (A) and (C) both have their risks, but
if they're non-default and everyone's willing to entertain a new arg
to the API method, sure. (A) seems more surprising as its failure mode
is subtler and does not mirror Hadoop semantics. I would have
suggested (C) as the fallback alternative. But I suppose you can
always make (C) happen yourself.

On Tue, Jun 3, 2014 at 4:33 AM, Kexin Xie  wrote:
> +1 on Option (B) with flag to allow semantics in (A) for back compatibility.
>
> Kexin
>
>
>
> On Tue, Jun 3, 2014 at 1:18 PM, Nicholas Chammas
>  wrote:
>>
>> On Mon, Jun 2, 2014 at 10:39 PM, Patrick Wendell 
>> wrote:
>>>
>>> (B) Semantics in Spark 1.0 and earlier:
>>
>>
>> Do you mean 1.0 and later?
>>
>> Option (B) with the exception-on-clobber sounds fine to me, btw. My use
>> pattern is probably common but not universal, and deleting user files is
>> indeed scary.
>>
>> Nick
>
>


Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Patrick Wendell
Good catch! Yes I meant 1.0 and later.

On Mon, Jun 2, 2014 at 8:33 PM, Kexin Xie  wrote:
> +1 on Option (B) with flag to allow semantics in (A) for back compatibility.
>
> Kexin
>
>
>
> On Tue, Jun 3, 2014 at 1:18 PM, Nicholas Chammas
>  wrote:
>>
>> On Mon, Jun 2, 2014 at 10:39 PM, Patrick Wendell 
>> wrote:
>>>
>>> (B) Semantics in Spark 1.0 and earlier:
>>
>>
>> Do you mean 1.0 and later?
>>
>> Option (B) with the exception-on-clobber sounds fine to me, btw. My use
>> pattern is probably common but not universal, and deleting user files is
>> indeed scary.
>>
>> Nick
>
>


Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Kexin Xie
+1 on Option (B) with flag to allow semantics in (A) for back compatibility.

Kexin


On Tue, Jun 3, 2014 at 1:18 PM, Nicholas Chammas  wrote:

> On Mon, Jun 2, 2014 at 10:39 PM, Patrick Wendell 
> wrote:
>
>> (B) Semantics in Spark 1.0 and earlier:
>
>
> Do you mean 1.0 and later?
>
> Option (B) with the exception-on-clobber sounds fine to me, btw. My use
> pattern is probably common but not universal, and deleting user files is
> indeed scary.
>
> Nick
>


Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Nicholas Chammas
On Mon, Jun 2, 2014 at 10:39 PM, Patrick Wendell  wrote:

> (B) Semantics in Spark 1.0 and earlier:


Do you mean 1.0 and later?

Option (B) with the exception-on-clobber sounds fine to me, btw. My use
pattern is probably common but not universal, and deleting user files is
indeed scary.

Nick


Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Nan Zhu
I remember that in the earlier version of that PR, I deleted files by calling 
HDFS API

we discussed and concluded that, it’s a bit scary to have something directly 
deleting user’s files in Spark

Best,  

--  
Nan Zhu


On Monday, June 2, 2014 at 10:39 PM, Patrick Wendell wrote:

> (A) Semantics in Spark 0.9 and earlier: Spark will ignore Hadoo's
> output format check and overwrite files in the destination directory.
> But it won't clobber the directory entirely. I.e. if the directory
> already had "part1" "part2" "part3" "part4" and you write a new job
> outputing only two files ("part1", "part2") then it would leave the
> other two files intact, confusingly.
>  
> (B) Semantics in Spark 1.0 and earlier: Runs Hadoop OutputFormat check
> which means the directory must not exist already or an excpetion is
> thrown.
>  
> (C) Semantics proposed by Nicholas Chammas in this thread (AFAIK):
> Spark will delete/clobber an existing destination directory if it
> exists, then fully over-write it with new data.
>  
> I'm fine to add a flag that allows (B) for backwards-compatibility
> reasons, but my point was I'd prefer not to have (C) even though I see
> some cases where it would be useful.
>  
> - Patrick
>  
> On Mon, Jun 2, 2014 at 4:25 PM, Sean Owen  (mailto:so...@cloudera.com)> wrote:
> > Is there a third way? Unless I miss something. Hadoop's OutputFormat
> > wants the target dir to not exist no matter what, so it's just a
> > question of whether Spark deletes it for you or errors.
> >  
> > On Tue, Jun 3, 2014 at 12:22 AM, Patrick Wendell  > (mailto:pwend...@gmail.com)> wrote:
> > > We can just add back a flag to make it backwards compatible - it was
> > > just missed during the original PR.
> > >  
> > > Adding a *third* set of "clobber" semantics, I'm slightly -1 on that
> > > for the following reasons:
> > >  
> > > 1. It's scary to have Spark recursively deleting user files, could
> > > easily lead to users deleting data by mistake if they don't understand
> > > the exact semantics.
> > > 2. It would introduce a third set of semantics here for saveAsXX...
> > > 3. It's trivial for users to implement this with two lines of code (if
> > > output dir exists, delete it) before calling saveAsHadoopFile.
> > >  
> > > - Patrick  



Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Patrick Wendell
(A) Semantics in Spark 0.9 and earlier: Spark will ignore Hadoo's
output format check and overwrite files in the destination directory.
But it won't clobber the directory entirely. I.e. if the directory
already had "part1" "part2" "part3" "part4" and you write a new job
outputing only two files ("part1", "part2") then it would leave the
other two files intact, confusingly.

(B) Semantics in Spark 1.0 and earlier: Runs Hadoop OutputFormat check
which means the directory must not exist already or an excpetion is
thrown.

(C) Semantics proposed by Nicholas Chammas in this thread (AFAIK):
Spark will delete/clobber an existing destination directory if it
exists, then fully over-write it with new data.

I'm fine to add a flag that allows (B) for backwards-compatibility
reasons, but my point was I'd prefer not to have (C) even though I see
some cases where it would be useful.

- Patrick

On Mon, Jun 2, 2014 at 4:25 PM, Sean Owen  wrote:
> Is there a third way? Unless I miss something. Hadoop's OutputFormat
> wants the target dir to not exist no matter what, so it's just a
> question of whether Spark deletes it for you or errors.
>
> On Tue, Jun 3, 2014 at 12:22 AM, Patrick Wendell  wrote:
>> We can just add back a flag to make it backwards compatible - it was
>> just missed during the original PR.
>>
>> Adding a *third* set of "clobber" semantics, I'm slightly -1 on that
>> for the following reasons:
>>
>> 1. It's scary to have Spark recursively deleting user files, could
>> easily lead to users deleting data by mistake if they don't understand
>> the exact semantics.
>> 2. It would introduce a third set of semantics here for saveAsXX...
>> 3. It's trivial for users to implement this with two lines of code (if
>> output dir exists, delete it) before calling saveAsHadoopFile.
>>
>> - Patrick
>>


Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Sean Owen
Is there a third way? Unless I miss something. Hadoop's OutputFormat
wants the target dir to not exist no matter what, so it's just a
question of whether Spark deletes it for you or errors.

On Tue, Jun 3, 2014 at 12:22 AM, Patrick Wendell  wrote:
> We can just add back a flag to make it backwards compatible - it was
> just missed during the original PR.
>
> Adding a *third* set of "clobber" semantics, I'm slightly -1 on that
> for the following reasons:
>
> 1. It's scary to have Spark recursively deleting user files, could
> easily lead to users deleting data by mistake if they don't understand
> the exact semantics.
> 2. It would introduce a third set of semantics here for saveAsXX...
> 3. It's trivial for users to implement this with two lines of code (if
> output dir exists, delete it) before calling saveAsHadoopFile.
>
> - Patrick
>


Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Patrick Wendell
We can just add back a flag to make it backwards compatible - it was
just missed during the original PR.

Adding a *third* set of "clobber" semantics, I'm slightly -1 on that
for the following reasons:

1. It's scary to have Spark recursively deleting user files, could
easily lead to users deleting data by mistake if they don't understand
the exact semantics.
2. It would introduce a third set of semantics here for saveAsXX...
3. It's trivial for users to implement this with two lines of code (if
output dir exists, delete it) before calling saveAsHadoopFile.

- Patrick

On Mon, Jun 2, 2014 at 2:49 PM, Nicholas Chammas
 wrote:
> Ah yes, this was indeed intended to have been taken care of:
>
>> add some new APIs with a flag for users to define whether he/she wants to
>> overwrite the directory: if the flag is set to true, then the output
>> directory is deleted first and then written into the new data to prevent the
>> output directory contains results from multiple rounds of running;
>
>
>
> On Mon, Jun 2, 2014 at 5:47 PM, Nan Zhu  wrote:
>>
>> I made the PR, the problem is …after many rounds of review, that
>> configuration part is missed….sorry about that
>>
>> I will fix it
>>
>> Best,
>>
>> --
>> Nan Zhu
>>
>> On Monday, June 2, 2014 at 5:13 PM, Pierre Borckmans wrote:
>>
>> I'm a bit confused because the PR mentioned by Patrick seems to adress all
>> these issues:
>>
>> https://github.com/apache/spark/commit/3a8b698e961ac05d9d53e2bbf0c2844fcb1010d1
>>
>> Was it not accepted? Or is the description of this PR not completely
>> implemented?
>>
>> Message sent from a mobile device - excuse typos and abbreviations
>>
>> Le 2 juin 2014 à 23:08, Nicholas Chammas  a
>> écrit :
>>
>> OK, thanks for confirming. Is there something we can do about that
>> leftover part- files problem in Spark, or is that for the Hadoop team?
>>
>>
>> 2014년 6월 2일 월요일, Aaron Davidson님이 작성한 메시지:
>>
>> Yes.
>>
>>
>> On Mon, Jun 2, 2014 at 1:23 PM, Nicholas Chammas
>>  wrote:
>>
>> So in summary:
>>
>> As of Spark 1.0.0, saveAsTextFile() will no longer clobber by default.
>> There is an open JIRA issue to add an option to allow clobbering.
>> Even when clobbering, part- files may be left over from previous saves,
>> which is dangerous.
>>
>> Is this correct?
>>
>>
>> On Mon, Jun 2, 2014 at 4:17 PM, Aaron Davidson  wrote:
>>
>> +1 please re-add this feature
>>
>>
>> On Mon, Jun 2, 2014 at 12:44 PM, Patrick Wendell 
>> wrote:
>>
>> Thanks for pointing that out. I've assigned you to SPARK-1677 (I think
>> I accidentally assigned myself way back when I created it). This
>> should be an easy fix.
>>
>> On Mon, Jun 2, 2014 at 12:19 PM, Nan Zhu  wrote:
>> > Hi, Patrick,
>> >
>> > I think https://issues.apache.org/jira/browse/SPARK-1677 is talking
>> > about
>> > the same thing?
>> >
>> > How about assigning it to me?
>> >
>> > I think I missed the configuration part in my previous commit, though I
>> > declared that in the PR description
>> >
>> > Best,
>> >
>> > --
>> > Nan Zhu
>> >
>> > On Monday, June 2, 2014 at 3:03 PM, Patrick Wendell wrote:
>> >
>> > Hey There,
>> >
>> > The issue was that the old behavior could cause users to silently
>> > overwrite data, which is pretty bad, so to be conservative we decided
>> > to enforce the same checks that Hadoop does.
>> >
>> > This was documented by this JIRA:
>> > https://issues.apache.org/jira/browse/SPARK-1100
>> >
>> > https://github.com/apache/spark/commit/3a8b698e961ac05d9d53e2bbf0c2844fcb1010d1
>> >
>> > However, it would be very easy to add an option that allows preserving
>> > the old behavior. Is anyone here interested in contributing that? I
>> > created a JIRA for it:
>> >
>> > https://issues.apache.org/jira/browse/SPARK-1993
>> >
>> > - Patrick
>> >
>> > On Mon, Jun 2, 2014 at 9:22 AM, Pierre Borckmans
>> >  wrote:
>> >
>> > Indeed, the behavior has changed for good or for bad. I mean, I agree
>> > with
>> > the danger you mention but I'm not sure it's happening like that. Isn't
>> > there a mechanism for overwrite in Hadoop that automatically removes
>> > part
>> > files, then writes a _temporary folder and then only the part files
>> > along
>> > with the _success folder.
>> >
>> > In any case this change of behavior should be documented IMO.
>> >
>> > Cheers
>> > Pierre
>> >
>> > Message sent from a mobile device - excuse typos and abbreviations
>> >
>> > Le 2 juin 2014 à 17:42, Nicholas Chammas  a
>> > écrit :
>> >
>> > What I've found using saveAsTextFile() against S3 (prior to Spark
>> > 1.0.0.) is
>> > that files get overwritten automatically. This is one danger to this
>> > though.
>> > If I save to a directory that already has 20 part- files, but this time
>> > around I'm only saving 15 part- files, then there will be 5 leftover
>> > part-
>> > files from the previous set mixed in with the 15 newer files. This is
>> > potentially dangerous.
>> >
>> > I haven't checked to see if this behavior has changed in 1.0.0. Are you
>>
>>
>


Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Nicholas Chammas
Ah yes, this was indeed intended to have been taken care of

:

add some new APIs with a flag for users to define whether he/she wants to
> overwrite the directory: if the flag is set to true, *then the output
> directory is deleted first* and then written into the new data to prevent
> the output directory contains results from multiple rounds of running;



On Mon, Jun 2, 2014 at 5:47 PM, Nan Zhu  wrote:

>  I made the PR, the problem is …after many rounds of review, that
> configuration part is missed….sorry about that
>
> I will fix it
>
> Best,
>
> --
> Nan Zhu
>
> On Monday, June 2, 2014 at 5:13 PM, Pierre Borckmans wrote:
>
> I'm a bit confused because the PR mentioned by Patrick seems to adress all
> these issues:
>
> https://github.com/apache/spark/commit/3a8b698e961ac05d9d53e2bbf0c2844fcb1010d1
>
> Was it not accepted? Or is the description of this PR not completely
> implemented?
>
> Message sent from a mobile device - excuse typos and abbreviations
>
> Le 2 juin 2014 à 23:08, Nicholas Chammas  a
> écrit :
>
> OK, thanks for confirming. Is there something we can do about that
> leftover part- files problem in Spark, or is that for the Hadoop team?
>
>
> 2014년 6월 2일 월요일, Aaron Davidson님이 작성한 메시지:
>
> Yes.
>
>
> On Mon, Jun 2, 2014 at 1:23 PM, Nicholas Chammas <
> nicholas.cham...@gmail.com> wrote:
>
> So in summary:
>
>- As of Spark 1.0.0, saveAsTextFile() will no longer clobber by
>default.
>- There is an open JIRA issue to add an option to allow clobbering.
>- Even when clobbering, part- files may be left over from previous
>saves, which is dangerous.
>
> Is this correct?
>
>
> On Mon, Jun 2, 2014 at 4:17 PM, Aaron Davidson  wrote:
>
> +1 please re-add this feature
>
>
> On Mon, Jun 2, 2014 at 12:44 PM, Patrick Wendell 
> wrote:
>
> Thanks for pointing that out. I've assigned you to SPARK-1677 (I think
> I accidentally assigned myself way back when I created it). This
> should be an easy fix.
>
> On Mon, Jun 2, 2014 at 12:19 PM, Nan Zhu  wrote:
> > Hi, Patrick,
> >
> > I think https://issues.apache.org/jira/browse/SPARK-1677 is talking
> about
> > the same thing?
> >
> > How about assigning it to me?
> >
> > I think I missed the configuration part in my previous commit, though I
> > declared that in the PR description
> >
> > Best,
> >
> > --
> > Nan Zhu
> >
> > On Monday, June 2, 2014 at 3:03 PM, Patrick Wendell wrote:
> >
> > Hey There,
> >
> > The issue was that the old behavior could cause users to silently
> > overwrite data, which is pretty bad, so to be conservative we decided
> > to enforce the same checks that Hadoop does.
> >
> > This was documented by this JIRA:
> > https://issues.apache.org/jira/browse/SPARK-1100
> >
> https://github.com/apache/spark/commit/3a8b698e961ac05d9d53e2bbf0c2844fcb1010d1
> >
> > However, it would be very easy to add an option that allows preserving
> > the old behavior. Is anyone here interested in contributing that? I
> > created a JIRA for it:
> >
> > https://issues.apache.org/jira/browse/SPARK-1993
> >
> > - Patrick
> >
> > On Mon, Jun 2, 2014 at 9:22 AM, Pierre Borckmans
> >  wrote:
> >
> > Indeed, the behavior has changed for good or for bad. I mean, I agree
> with
> > the danger you mention but I'm not sure it's happening like that. Isn't
> > there a mechanism for overwrite in Hadoop that automatically removes part
> > files, then writes a _temporary folder and then only the part files along
> > with the _success folder.
> >
> > In any case this change of behavior should be documented IMO.
> >
> > Cheers
> > Pierre
> >
> > Message sent from a mobile device - excuse typos and abbreviations
> >
> > Le 2 juin 2014 à 17:42, Nicholas Chammas  a
> > écrit :
> >
> > What I've found using saveAsTextFile() against S3 (prior to Spark
> 1.0.0.) is
> > that files get overwritten automatically. This is one danger to this
> though.
> > If I save to a directory that already has 20 part- files, but this time
> > around I'm only saving 15 part- files, then there will be 5 leftover
> part-
> > files from the previous set mixed in with the 15 newer files. This is
> > potentially dangerous.
> >
> > I haven't checked to see if this behavior has changed in 1.0.0. Are you
>
>
>


Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Nan Zhu
I made the PR, the problem is …after many rounds of review, that configuration 
part is missed….sorry about that  

I will fix it  

Best,  

--  
Nan Zhu


On Monday, June 2, 2014 at 5:13 PM, Pierre Borckmans wrote:

> I'm a bit confused because the PR mentioned by Patrick seems to adress all 
> these issues:
> https://github.com/apache/spark/commit/3a8b698e961ac05d9d53e2bbf0c2844fcb1010d1
>  
> Was it not accepted? Or is the description of this PR not completely 
> implemented?
>  
> Message sent from a mobile device - excuse typos and abbreviations
>  
> Le 2 juin 2014 à 23:08, Nicholas Chammas  (mailto:nicholas.cham...@gmail.com)> a écrit :
>  
> > OK, thanks for confirming. Is there something we can do about that leftover 
> > part- files problem in Spark, or is that for the Hadoop team?
> >  
> >  
> > 2014년 6월 2일 월요일, Aaron Davidson > (mailto:ilike...@gmail.com)>님이 작성한 메시지:
> > > Yes.
> > >  
> > >  
> > > On Mon, Jun 2, 2014 at 1:23 PM, Nicholas Chammas 
> > >  wrote:
> > > > So in summary:
> > > > As of Spark 1.0.0, saveAsTextFile() will no longer clobber by default.
> > > > There is an open JIRA issue to add an option to allow clobbering.
> > > > Even when clobbering, part- files may be left over from previous saves, 
> > > > which is dangerous.
> > > >  
> > > > Is this correct?
> > > >  
> > > >  
> > > >  
> > > >  
> > > > On Mon, Jun 2, 2014 at 4:17 PM, Aaron Davidson  
> > > > wrote:
> > > > > +1 please re-add this feature
> > > > >  
> > > > >  
> > > > > On Mon, Jun 2, 2014 at 12:44 PM, Patrick Wendell  
> > > > > wrote:
> > > > > > Thanks for pointing that out. I've assigned you to SPARK-1677 (I 
> > > > > > think
> > > > > > I accidentally assigned myself way back when I created it). This
> > > > > > should be an easy fix.
> > > > > >  
> > > > > > On Mon, Jun 2, 2014 at 12:19 PM, Nan Zhu  
> > > > > > wrote:
> > > > > > > Hi, Patrick,
> > > > > > >
> > > > > > > I think https://issues.apache.org/jira/browse/SPARK-1677 is 
> > > > > > > talking about
> > > > > > > the same thing?
> > > > > > >
> > > > > > > How about assigning it to me?
> > > > > > >
> > > > > > > I think I missed the configuration part in my previous commit, 
> > > > > > > though I
> > > > > > > declared that in the PR description
> > > > > > >
> > > > > > > Best,
> > > > > > >
> > > > > > > --
> > > > > > > Nan Zhu
> > > > > > >
> > > > > > > On Monday, June 2, 2014 at 3:03 PM, Patrick Wendell wrote:
> > > > > > >
> > > > > > > Hey There,
> > > > > > >
> > > > > > > The issue was that the old behavior could cause users to silently
> > > > > > > overwrite data, which is pretty bad, so to be conservative we 
> > > > > > > decided
> > > > > > > to enforce the same checks that Hadoop does.
> > > > > > >
> > > > > > > This was documented by this JIRA:
> > > > > > > https://issues.apache.org/jira/browse/SPARK-1100
> > > > > > > https://github.com/apache/spark/commit/3a8b698e961ac05d9d53e2bbf0c2844fcb1010d1
> > > > > > >
> > > > > > > However, it would be very easy to add an option that allows 
> > > > > > > preserving
> > > > > > > the old behavior. Is anyone here interested in contributing that? 
> > > > > > > I
> > > > > > > created a JIRA for it:
> > > > > > >
> > > > > > > https://issues.apache.org/jira/browse/SPARK-1993
> > > > > > >
> > > > > > > - Patrick
> > > > > > >
> > > > > > > On Mon, Jun 2, 2014 at 9:22 AM, Pierre Borckmans
> > > > > > >  wrote:
> > > > > > >
> > > > > > > Indeed, the behavior has changed for good or for bad. I mean, I 
> > > > > > > agree with
> > > > > > > the danger you mention but I'm not sure it's happening like that. 
> > > > > > > Isn't
> > > > > > > there a mechanism for overwrite in Hadoop that automatically 
> > > > > > > removes part
> > > > > > > files, then writes a _temporary folder and then only the part 
> > > > > > > files along
> > > > > > > with the _success folder.
> > > > > > >
> > > > > > > In any case this change of behavior should be documented IMO.
> > > > > > >
> > > > > > > Cheers
> > > > > > > Pierre
> > > > > > >
> > > > > > > Message sent from a mobile device - excuse typos and abbreviations
> > > > > > >
> > > > > > > Le 2 juin 2014 à 17:42, Nicholas Chammas 
> > > > > > >  a
> > > > > > > écrit :
> > > > > > >
> > > > > > > What I've found using saveAsTextFile() against S3 (prior to Spark 
> > > > > > > 1.0.0.) is
> > > > > > > that files get overwritten automatically. This is one danger to 
> > > > > > > this though.
> > > > > > > If I save to a directory that already has 20 part- files, but 
> > > > > > > this time
> > > > > > > around I'm only saving 15 part- files, then there will be 5 
> > > > > > > leftover part-
> > > > > > > files from the previous set mixed in with the 15 newer files. 
> > > > > > > This is
> > > > > > > potentially dangerous.
> > > > > > >
> > > > > > > I haven't checked to see if this behavior has changed in 1.0.0. 
> > > > > > > Are you



Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Nicholas Chammas
Fair enough. That rationale makes sense.

I would prefer that a Spark clobber option also delete the destination
files, but as long as it's a non-default option I can see the "caller
beware" side of that argument as well.

Nick


2014년 6월 2일 월요일, Sean Owen님이 작성한 메시지:

> I assume the idea is for Spark to "rm -r dir/", which would clean out
> everything that was there before. It's just doing this instead of the
> caller. Hadoop still won't let you write into a location that already
> exists regardless, and part of that is for this reason that you might
> end up with files mixed-up from different jobs.
>
> This doesn't need a change to Hadoop and probably shouldn't; it's a
> change to semantics provided by Spark to do the delete for you if you
> set a flag. Viewed that way, meh, seems like the caller could just do
> that themselves rather than expand the Spark API (via a utility method
> if you like), but I can see it both ways. Caller beware.
>
> On Mon, Jun 2, 2014 at 10:08 PM, Nicholas Chammas
> > wrote:
> > OK, thanks for confirming. Is there something we can do about that
> leftover
> > part- files problem in Spark, or is that for the Hadoop team?
> >
> >
> > 2014년 6월 2일 월요일, Aaron Davidson>님이
> 작성한 메시지:
> >
> >> Yes.
> >>
> >>
> >> On Mon, Jun 2, 2014 at 1:23 PM, Nicholas Chammas
> >> > wrote:
> >>
> >> So in summary:
> >>
> >> As of Spark 1.0.0, saveAsTextFile() will no longer clobber by default.
> >> There is an open JIRA issue to add an option to allow clobbering.
> >> Even when clobbering, part- files may be left over from previous saves,
> >> which is dangerous.
> >>
> >> Is this correct?
>


Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Sean Owen
I assume the idea is for Spark to "rm -r dir/", which would clean out
everything that was there before. It's just doing this instead of the
caller. Hadoop still won't let you write into a location that already
exists regardless, and part of that is for this reason that you might
end up with files mixed-up from different jobs.

This doesn't need a change to Hadoop and probably shouldn't; it's a
change to semantics provided by Spark to do the delete for you if you
set a flag. Viewed that way, meh, seems like the caller could just do
that themselves rather than expand the Spark API (via a utility method
if you like), but I can see it both ways. Caller beware.

On Mon, Jun 2, 2014 at 10:08 PM, Nicholas Chammas
 wrote:
> OK, thanks for confirming. Is there something we can do about that leftover
> part- files problem in Spark, or is that for the Hadoop team?
>
>
> 2014년 6월 2일 월요일, Aaron Davidson님이 작성한 메시지:
>
>> Yes.
>>
>>
>> On Mon, Jun 2, 2014 at 1:23 PM, Nicholas Chammas
>>  wrote:
>>
>> So in summary:
>>
>> As of Spark 1.0.0, saveAsTextFile() will no longer clobber by default.
>> There is an open JIRA issue to add an option to allow clobbering.
>> Even when clobbering, part- files may be left over from previous saves,
>> which is dangerous.
>>
>> Is this correct?


Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Pierre Borckmans
I'm a bit confused because the PR mentioned by Patrick seems to adress all 
these issues:
https://github.com/apache/spark/commit/3a8b698e961ac05d9d53e2bbf0c2844fcb1010d1

Was it not accepted? Or is the description of this PR not completely 
implemented?

Message sent from a mobile device - excuse typos and abbreviations

> Le 2 juin 2014 à 23:08, Nicholas Chammas  a écrit 
> :
> 
> OK, thanks for confirming. Is there something we can do about that leftover 
> part- files problem in Spark, or is that for the Hadoop team?
> 
> 
> 2014년 6월 2일 월요일, Aaron Davidson님이 작성한 메시지:
>> Yes.
>> 
>> 
>> On Mon, Jun 2, 2014 at 1:23 PM, Nicholas Chammas 
>>  wrote:
>> So in summary:
>> As of Spark 1.0.0, saveAsTextFile() will no longer clobber by default.
>> There is an open JIRA issue to add an option to allow clobbering.
>> Even when clobbering, part- files may be left over from previous saves, 
>> which is dangerous.
>> Is this correct?
>> 
>> 
>> On Mon, Jun 2, 2014 at 4:17 PM, Aaron Davidson  wrote:
>> +1 please re-add this feature
>> 
>> 
>> On Mon, Jun 2, 2014 at 12:44 PM, Patrick Wendell  wrote:
>> Thanks for pointing that out. I've assigned you to SPARK-1677 (I think
>> I accidentally assigned myself way back when I created it). This
>> should be an easy fix.
>> 
>> On Mon, Jun 2, 2014 at 12:19 PM, Nan Zhu  wrote:
>> > Hi, Patrick,
>> >
>> > I think https://issues.apache.org/jira/browse/SPARK-1677 is talking about
>> > the same thing?
>> >
>> > How about assigning it to me?
>> >
>> > I think I missed the configuration part in my previous commit, though I
>> > declared that in the PR description
>> >
>> > Best,
>> >
>> > --
>> > Nan Zhu
>> >
>> > On Monday, June 2, 2014 at 3:03 PM, Patrick Wendell wrote:
>> >
>> > Hey There,
>> >
>> > The issue was that the old behavior could cause users to silently
>> > overwrite data, which is pretty bad, so to be conservative we decided
>> > to enforce the same checks that Hadoop does.
>> >
>> > This was documented by this JIRA:
>> > https://issues.apache.org/jira/browse/SPARK-1100
>> > https://github.com/apache/spark/commit/3a8b698e961ac05d9d53e2bbf0c2844fcb1010d1
>> >
>> > However, it would be very easy to add an option that allows preserving
>> > the old behavior. Is anyone here interested in contributing that? I
>> > created a JIRA for it:
>> >
>> > https://issues.apache.org/jira/browse/SPARK-1993
>> >
>> > - Patrick
>> >
>> > On Mon, Jun 2, 2014 at 9:22 AM, Pierre Borckmans
>> >  wrote:
>> >
>> > Indeed, the behavior has changed for good or for bad. I mean, I agree with
>> > the danger you mention but I'm not sure it's happening like that. Isn't
>> > there a mechanism for overwrite in Hadoop that automatically removes part
>> > files, then writes a _temporary folder and then only the part files along
>> > with the _success folder.
>> >
>> > In any case this change of behavior should be documented IMO.
>> >
>> > Cheers
>> > Pierre
>> >
>> > Message sent from a mobile device - excuse typos and abbreviations
>> >
>> > Le 2 juin 2014 à 17:42, Nicholas Chammas  a
>> > écrit :
>> >
>> > What I've found using saveAsTextFile() against S3 (prior to Spark 1.0.0.) 
>> > is
>> > that files get overwritten automatically. This is one danger to this 
>> > though.
>> > If I save to a directory that already has 20 part- files, but this time
>> > around I'm only saving 15 part- files, then there will be 5 leftover part-
>> > files from the previous set mixed in with the 15 newer files. This is
>> > potentially dangerous.
>> >
>> > I haven't checked to see if this behavior has changed in 1.0.0. Are you


Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Nicholas Chammas
OK, thanks for confirming. Is there something we can do about that leftover
part- files problem in Spark, or is that for the Hadoop team?


2014년 6월 2일 월요일, Aaron Davidson님이 작성한 메시지:

> Yes.
>
>
> On Mon, Jun 2, 2014 at 1:23 PM, Nicholas Chammas <
> nicholas.cham...@gmail.com> wrote:
>
> So in summary:
>
>- As of Spark 1.0.0, saveAsTextFile() will no longer clobber by
>default.
>- There is an open JIRA issue to add an option to allow clobbering.
>- Even when clobbering, part- files may be left over from previous
>saves, which is dangerous.
>
> Is this correct?
>
>
> On Mon, Jun 2, 2014 at 4:17 PM, Aaron Davidson  wrote:
>
> +1 please re-add this feature
>
>
> On Mon, Jun 2, 2014 at 12:44 PM, Patrick Wendell 
> wrote:
>
> Thanks for pointing that out. I've assigned you to SPARK-1677 (I think
> I accidentally assigned myself way back when I created it). This
> should be an easy fix.
>
> On Mon, Jun 2, 2014 at 12:19 PM, Nan Zhu  wrote:
> > Hi, Patrick,
> >
> > I think https://issues.apache.org/jira/browse/SPARK-1677 is talking
> about
> > the same thing?
> >
> > How about assigning it to me?
> >
> > I think I missed the configuration part in my previous commit, though I
> > declared that in the PR description
> >
> > Best,
> >
> > --
> > Nan Zhu
> >
> > On Monday, June 2, 2014 at 3:03 PM, Patrick Wendell wrote:
> >
> > Hey There,
> >
> > The issue was that the old behavior could cause users to silently
> > overwrite data, which is pretty bad, so to be conservative we decided
> > to enforce the same checks that Hadoop does.
> >
> > This was documented by this JIRA:
> > https://issues.apache.org/jira/browse/SPARK-1100
> >
> https://github.com/apache/spark/commit/3a8b698e961ac05d9d53e2bbf0c2844fcb1010d1
> >
> > However, it would be very easy to add an option that allows preserving
> > the old behavior. Is anyone here interested in contributing that? I
> > created a JIRA for it:
> >
> > https://issues.apache.org/jira/browse/SPARK-1993
> >
> > - Patrick
> >
> > On Mon, Jun 2, 2014 at 9:22 AM, Pierre Borckmans
> >  wrote:
> >
> > Indeed, the behavior has changed for good or for bad. I mean, I agree
> with
> > the danger you mention but I'm not sure it's happening like that. Isn't
> > there a mechanism for overwrite in Hadoop that automatically removes part
> > files, then writes a _temporary folder and then only the part files along
> > with the _success folder.
> >
> > In any case this change of behavior should be documented IMO.
> >
> > Cheers
> > Pierre
> >
> > Message sent from a mobile device - excuse typos and abbreviations
> >
> > Le 2 juin 2014 à 17:42, Nicholas Chammas  a
> > écrit :
> >
> > What I've found using saveAsTextFile() against S3 (prior to Spark
> 1.0.0.) is
> > that files get overwritten automatically. This is one danger to this
> though.
> > If I save to a directory that already has 20 part- files, but this time
> > around I'm only saving 15 part- files, then there will be 5 leftover
> part-
> > files from the previous set mixed in with the 15 newer files. This is
> > potentially dangerous.
> >
> > I haven't checked to see if this behavior has changed in 1.0.0. Are you
>
>


Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Aaron Davidson
Yes.


On Mon, Jun 2, 2014 at 1:23 PM, Nicholas Chammas  wrote:

> So in summary:
>
>- As of Spark 1.0.0, saveAsTextFile() will no longer clobber by
>default.
>- There is an open JIRA issue to add an option to allow clobbering.
>- Even when clobbering, part- files may be left over from previous
>saves, which is dangerous.
>
> Is this correct?
>
>
> On Mon, Jun 2, 2014 at 4:17 PM, Aaron Davidson  wrote:
>
>> +1 please re-add this feature
>>
>>
>> On Mon, Jun 2, 2014 at 12:44 PM, Patrick Wendell 
>> wrote:
>>
>>> Thanks for pointing that out. I've assigned you to SPARK-1677 (I think
>>> I accidentally assigned myself way back when I created it). This
>>> should be an easy fix.
>>>
>>> On Mon, Jun 2, 2014 at 12:19 PM, Nan Zhu  wrote:
>>> > Hi, Patrick,
>>> >
>>> > I think https://issues.apache.org/jira/browse/SPARK-1677 is talking
>>> about
>>> > the same thing?
>>> >
>>> > How about assigning it to me?
>>> >
>>> > I think I missed the configuration part in my previous commit, though I
>>> > declared that in the PR description
>>> >
>>> > Best,
>>> >
>>> > --
>>> > Nan Zhu
>>> >
>>> > On Monday, June 2, 2014 at 3:03 PM, Patrick Wendell wrote:
>>> >
>>> > Hey There,
>>> >
>>> > The issue was that the old behavior could cause users to silently
>>> > overwrite data, which is pretty bad, so to be conservative we decided
>>> > to enforce the same checks that Hadoop does.
>>> >
>>> > This was documented by this JIRA:
>>> > https://issues.apache.org/jira/browse/SPARK-1100
>>> >
>>> https://github.com/apache/spark/commit/3a8b698e961ac05d9d53e2bbf0c2844fcb1010d1
>>> >
>>> > However, it would be very easy to add an option that allows preserving
>>> > the old behavior. Is anyone here interested in contributing that? I
>>> > created a JIRA for it:
>>> >
>>> > https://issues.apache.org/jira/browse/SPARK-1993
>>> >
>>> > - Patrick
>>> >
>>> > On Mon, Jun 2, 2014 at 9:22 AM, Pierre Borckmans
>>> >  wrote:
>>> >
>>> > Indeed, the behavior has changed for good or for bad. I mean, I agree
>>> with
>>> > the danger you mention but I'm not sure it's happening like that. Isn't
>>> > there a mechanism for overwrite in Hadoop that automatically removes
>>> part
>>> > files, then writes a _temporary folder and then only the part files
>>> along
>>> > with the _success folder.
>>> >
>>> > In any case this change of behavior should be documented IMO.
>>> >
>>> > Cheers
>>> > Pierre
>>> >
>>> > Message sent from a mobile device - excuse typos and abbreviations
>>> >
>>> > Le 2 juin 2014 à 17:42, Nicholas Chammas 
>>> a
>>> > écrit :
>>> >
>>> > What I've found using saveAsTextFile() against S3 (prior to Spark
>>> 1.0.0.) is
>>> > that files get overwritten automatically. This is one danger to this
>>> though.
>>> > If I save to a directory that already has 20 part- files, but this time
>>> > around I'm only saving 15 part- files, then there will be 5 leftover
>>> part-
>>> > files from the previous set mixed in with the 15 newer files. This is
>>> > potentially dangerous.
>>> >
>>> > I haven't checked to see if this behavior has changed in 1.0.0. Are you
>>> > saying it has, Pierre?
>>> >
>>> > On Mon, Jun 2, 2014 at 9:41 AM, Pierre B
>>> > [pierre.borckm...@realimpactanalytics.com](mailto:
>>> pierre.borckm...@realimpactanalytics.com)
>>> > wrote:
>>> >
>>> >
>>> > Hi Michaël,
>>> >
>>> > Thanks for this. We could indeed do that.
>>> >
>>> > But I guess the question is more about the change of behaviour from
>>> 0.9.1
>>> > to
>>> > 1.0.0.
>>> > We never had to care about that in previous versions.
>>> >
>>> > Does that mean we have to manually remove existing files or is there a
>>> way
>>> > to "aumotically" overwrite when using saveAsTextFile?
>>> >
>>> >
>>> >
>>> > --
>>> > View this message in context:
>>> >
>>> http://apache-spark-user-list.1001560.n3.nabble.com/How-can-I-make-Spark-1-0-saveAsTextFile-to-overwrite-existing-file-tp6696p6700.html
>>> > Sent from the Apache Spark User List mailing list archive at
>>> Nabble.com.
>>> >
>>> >
>>>
>>
>>
>


Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Nicholas Chammas
So in summary:

   - As of Spark 1.0.0, saveAsTextFile() will no longer clobber by default.
   - There is an open JIRA issue to add an option to allow clobbering.
   - Even when clobbering, part- files may be left over from previous
   saves, which is dangerous.

Is this correct?


On Mon, Jun 2, 2014 at 4:17 PM, Aaron Davidson  wrote:

> +1 please re-add this feature
>
>
> On Mon, Jun 2, 2014 at 12:44 PM, Patrick Wendell 
> wrote:
>
>> Thanks for pointing that out. I've assigned you to SPARK-1677 (I think
>> I accidentally assigned myself way back when I created it). This
>> should be an easy fix.
>>
>> On Mon, Jun 2, 2014 at 12:19 PM, Nan Zhu  wrote:
>> > Hi, Patrick,
>> >
>> > I think https://issues.apache.org/jira/browse/SPARK-1677 is talking
>> about
>> > the same thing?
>> >
>> > How about assigning it to me?
>> >
>> > I think I missed the configuration part in my previous commit, though I
>> > declared that in the PR description
>> >
>> > Best,
>> >
>> > --
>> > Nan Zhu
>> >
>> > On Monday, June 2, 2014 at 3:03 PM, Patrick Wendell wrote:
>> >
>> > Hey There,
>> >
>> > The issue was that the old behavior could cause users to silently
>> > overwrite data, which is pretty bad, so to be conservative we decided
>> > to enforce the same checks that Hadoop does.
>> >
>> > This was documented by this JIRA:
>> > https://issues.apache.org/jira/browse/SPARK-1100
>> >
>> https://github.com/apache/spark/commit/3a8b698e961ac05d9d53e2bbf0c2844fcb1010d1
>> >
>> > However, it would be very easy to add an option that allows preserving
>> > the old behavior. Is anyone here interested in contributing that? I
>> > created a JIRA for it:
>> >
>> > https://issues.apache.org/jira/browse/SPARK-1993
>> >
>> > - Patrick
>> >
>> > On Mon, Jun 2, 2014 at 9:22 AM, Pierre Borckmans
>> >  wrote:
>> >
>> > Indeed, the behavior has changed for good or for bad. I mean, I agree
>> with
>> > the danger you mention but I'm not sure it's happening like that. Isn't
>> > there a mechanism for overwrite in Hadoop that automatically removes
>> part
>> > files, then writes a _temporary folder and then only the part files
>> along
>> > with the _success folder.
>> >
>> > In any case this change of behavior should be documented IMO.
>> >
>> > Cheers
>> > Pierre
>> >
>> > Message sent from a mobile device - excuse typos and abbreviations
>> >
>> > Le 2 juin 2014 à 17:42, Nicholas Chammas  a
>> > écrit :
>> >
>> > What I've found using saveAsTextFile() against S3 (prior to Spark
>> 1.0.0.) is
>> > that files get overwritten automatically. This is one danger to this
>> though.
>> > If I save to a directory that already has 20 part- files, but this time
>> > around I'm only saving 15 part- files, then there will be 5 leftover
>> part-
>> > files from the previous set mixed in with the 15 newer files. This is
>> > potentially dangerous.
>> >
>> > I haven't checked to see if this behavior has changed in 1.0.0. Are you
>> > saying it has, Pierre?
>> >
>> > On Mon, Jun 2, 2014 at 9:41 AM, Pierre B
>> > [pierre.borckm...@realimpactanalytics.com](mailto:
>> pierre.borckm...@realimpactanalytics.com)
>> > wrote:
>> >
>> >
>> > Hi Michaël,
>> >
>> > Thanks for this. We could indeed do that.
>> >
>> > But I guess the question is more about the change of behaviour from
>> 0.9.1
>> > to
>> > 1.0.0.
>> > We never had to care about that in previous versions.
>> >
>> > Does that mean we have to manually remove existing files or is there a
>> way
>> > to "aumotically" overwrite when using saveAsTextFile?
>> >
>> >
>> >
>> > --
>> > View this message in context:
>> >
>> http://apache-spark-user-list.1001560.n3.nabble.com/How-can-I-make-Spark-1-0-saveAsTextFile-to-overwrite-existing-file-tp6696p6700.html
>> > Sent from the Apache Spark User List mailing list archive at Nabble.com.
>> >
>> >
>>
>
>


Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Aaron Davidson
+1 please re-add this feature


On Mon, Jun 2, 2014 at 12:44 PM, Patrick Wendell  wrote:

> Thanks for pointing that out. I've assigned you to SPARK-1677 (I think
> I accidentally assigned myself way back when I created it). This
> should be an easy fix.
>
> On Mon, Jun 2, 2014 at 12:19 PM, Nan Zhu  wrote:
> > Hi, Patrick,
> >
> > I think https://issues.apache.org/jira/browse/SPARK-1677 is talking
> about
> > the same thing?
> >
> > How about assigning it to me?
> >
> > I think I missed the configuration part in my previous commit, though I
> > declared that in the PR description
> >
> > Best,
> >
> > --
> > Nan Zhu
> >
> > On Monday, June 2, 2014 at 3:03 PM, Patrick Wendell wrote:
> >
> > Hey There,
> >
> > The issue was that the old behavior could cause users to silently
> > overwrite data, which is pretty bad, so to be conservative we decided
> > to enforce the same checks that Hadoop does.
> >
> > This was documented by this JIRA:
> > https://issues.apache.org/jira/browse/SPARK-1100
> >
> https://github.com/apache/spark/commit/3a8b698e961ac05d9d53e2bbf0c2844fcb1010d1
> >
> > However, it would be very easy to add an option that allows preserving
> > the old behavior. Is anyone here interested in contributing that? I
> > created a JIRA for it:
> >
> > https://issues.apache.org/jira/browse/SPARK-1993
> >
> > - Patrick
> >
> > On Mon, Jun 2, 2014 at 9:22 AM, Pierre Borckmans
> >  wrote:
> >
> > Indeed, the behavior has changed for good or for bad. I mean, I agree
> with
> > the danger you mention but I'm not sure it's happening like that. Isn't
> > there a mechanism for overwrite in Hadoop that automatically removes part
> > files, then writes a _temporary folder and then only the part files along
> > with the _success folder.
> >
> > In any case this change of behavior should be documented IMO.
> >
> > Cheers
> > Pierre
> >
> > Message sent from a mobile device - excuse typos and abbreviations
> >
> > Le 2 juin 2014 à 17:42, Nicholas Chammas  a
> > écrit :
> >
> > What I've found using saveAsTextFile() against S3 (prior to Spark
> 1.0.0.) is
> > that files get overwritten automatically. This is one danger to this
> though.
> > If I save to a directory that already has 20 part- files, but this time
> > around I'm only saving 15 part- files, then there will be 5 leftover
> part-
> > files from the previous set mixed in with the 15 newer files. This is
> > potentially dangerous.
> >
> > I haven't checked to see if this behavior has changed in 1.0.0. Are you
> > saying it has, Pierre?
> >
> > On Mon, Jun 2, 2014 at 9:41 AM, Pierre B
> > [pierre.borckm...@realimpactanalytics.com](mailto:
> pierre.borckm...@realimpactanalytics.com)
> > wrote:
> >
> >
> > Hi Michaël,
> >
> > Thanks for this. We could indeed do that.
> >
> > But I guess the question is more about the change of behaviour from 0.9.1
> > to
> > 1.0.0.
> > We never had to care about that in previous versions.
> >
> > Does that mean we have to manually remove existing files or is there a
> way
> > to "aumotically" overwrite when using saveAsTextFile?
> >
> >
> >
> > --
> > View this message in context:
> >
> http://apache-spark-user-list.1001560.n3.nabble.com/How-can-I-make-Spark-1-0-saveAsTextFile-to-overwrite-existing-file-tp6696p6700.html
> > Sent from the Apache Spark User List mailing list archive at Nabble.com.
> >
> >
>


Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Patrick Wendell
Thanks for pointing that out. I've assigned you to SPARK-1677 (I think
I accidentally assigned myself way back when I created it). This
should be an easy fix.

On Mon, Jun 2, 2014 at 12:19 PM, Nan Zhu  wrote:
> Hi, Patrick,
>
> I think https://issues.apache.org/jira/browse/SPARK-1677 is talking about
> the same thing?
>
> How about assigning it to me?
>
> I think I missed the configuration part in my previous commit, though I
> declared that in the PR description
>
> Best,
>
> --
> Nan Zhu
>
> On Monday, June 2, 2014 at 3:03 PM, Patrick Wendell wrote:
>
> Hey There,
>
> The issue was that the old behavior could cause users to silently
> overwrite data, which is pretty bad, so to be conservative we decided
> to enforce the same checks that Hadoop does.
>
> This was documented by this JIRA:
> https://issues.apache.org/jira/browse/SPARK-1100
> https://github.com/apache/spark/commit/3a8b698e961ac05d9d53e2bbf0c2844fcb1010d1
>
> However, it would be very easy to add an option that allows preserving
> the old behavior. Is anyone here interested in contributing that? I
> created a JIRA for it:
>
> https://issues.apache.org/jira/browse/SPARK-1993
>
> - Patrick
>
> On Mon, Jun 2, 2014 at 9:22 AM, Pierre Borckmans
>  wrote:
>
> Indeed, the behavior has changed for good or for bad. I mean, I agree with
> the danger you mention but I'm not sure it's happening like that. Isn't
> there a mechanism for overwrite in Hadoop that automatically removes part
> files, then writes a _temporary folder and then only the part files along
> with the _success folder.
>
> In any case this change of behavior should be documented IMO.
>
> Cheers
> Pierre
>
> Message sent from a mobile device - excuse typos and abbreviations
>
> Le 2 juin 2014 à 17:42, Nicholas Chammas  a
> écrit :
>
> What I've found using saveAsTextFile() against S3 (prior to Spark 1.0.0.) is
> that files get overwritten automatically. This is one danger to this though.
> If I save to a directory that already has 20 part- files, but this time
> around I'm only saving 15 part- files, then there will be 5 leftover part-
> files from the previous set mixed in with the 15 newer files. This is
> potentially dangerous.
>
> I haven't checked to see if this behavior has changed in 1.0.0. Are you
> saying it has, Pierre?
>
> On Mon, Jun 2, 2014 at 9:41 AM, Pierre B
> [pierre.borckm...@realimpactanalytics.com](mailto:pierre.borckm...@realimpactanalytics.com)
> wrote:
>
>
> Hi Michaël,
>
> Thanks for this. We could indeed do that.
>
> But I guess the question is more about the change of behaviour from 0.9.1
> to
> 1.0.0.
> We never had to care about that in previous versions.
>
> Does that mean we have to manually remove existing files or is there a way
> to "aumotically" overwrite when using saveAsTextFile?
>
>
>
> --
> View this message in context:
> http://apache-spark-user-list.1001560.n3.nabble.com/How-can-I-make-Spark-1-0-saveAsTextFile-to-overwrite-existing-file-tp6696p6700.html
> Sent from the Apache Spark User List mailing list archive at Nabble.com.
>
>


Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Nan Zhu
Hi, Patrick,   

I think https://issues.apache.org/jira/browse/SPARK-1677 is talking about the 
same thing?

How about assigning it to me?  

I think I missed the configuration part in my previous commit, though I 
declared that in the PR description….

Best,  

--  
Nan Zhu


On Monday, June 2, 2014 at 3:03 PM, Patrick Wendell wrote:

> Hey There,
>  
> The issue was that the old behavior could cause users to silently
> overwrite data, which is pretty bad, so to be conservative we decided
> to enforce the same checks that Hadoop does.
>  
> This was documented by this JIRA:
> https://issues.apache.org/jira/browse/SPARK-1100
> https://github.com/apache/spark/commit/3a8b698e961ac05d9d53e2bbf0c2844fcb1010d1
>  
> However, it would be very easy to add an option that allows preserving
> the old behavior. Is anyone here interested in contributing that? I
> created a JIRA for it:
>  
> https://issues.apache.org/jira/browse/SPARK-1993
>  
> - Patrick
>  
> On Mon, Jun 2, 2014 at 9:22 AM, Pierre Borckmans
>  (mailto:pierre.borckm...@realimpactanalytics.com)> wrote:
> > Indeed, the behavior has changed for good or for bad. I mean, I agree with
> > the danger you mention but I'm not sure it's happening like that. Isn't
> > there a mechanism for overwrite in Hadoop that automatically removes part
> > files, then writes a _temporary folder and then only the part files along
> > with the _success folder.
> >  
> > In any case this change of behavior should be documented IMO.
> >  
> > Cheers
> > Pierre
> >  
> > Message sent from a mobile device - excuse typos and abbreviations
> >  
> > Le 2 juin 2014 à 17:42, Nicholas Chammas  > (mailto:nicholas.cham...@gmail.com)> a
> > écrit :
> >  
> > What I've found using saveAsTextFile() against S3 (prior to Spark 1.0.0.) is
> > that files get overwritten automatically. This is one danger to this though.
> > If I save to a directory that already has 20 part- files, but this time
> > around I'm only saving 15 part- files, then there will be 5 leftover part-
> > files from the previous set mixed in with the 15 newer files. This is
> > potentially dangerous.
> >  
> > I haven't checked to see if this behavior has changed in 1.0.0. Are you
> > saying it has, Pierre?
> >  
> > On Mon, Jun 2, 2014 at 9:41 AM, Pierre B
> > [pierre.borckm...@realimpactanalytics.com](mailto:pierre.borckm...@realimpactanalytics.com)
> > wrote:
> > >  
> > > Hi Michaël,
> > >  
> > > Thanks for this. We could indeed do that.
> > >  
> > > But I guess the question is more about the change of behaviour from 0.9.1
> > > to
> > > 1.0.0.
> > > We never had to care about that in previous versions.
> > >  
> > > Does that mean we have to manually remove existing files or is there a way
> > > to "aumotically" overwrite when using saveAsTextFile?
> > >  
> > >  
> > >  
> > > --
> > > View this message in context:
> > > http://apache-spark-user-list.1001560.n3.nabble.com/How-can-I-make-Spark-1-0-saveAsTextFile-to-overwrite-existing-file-tp6696p6700.html
> > > Sent from the Apache Spark User List mailing list archive at Nabble.com 
> > > (http://Nabble.com).
> > >  
> >  
> >  
>  
>  
>  




Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Patrick Wendell
Hey There,

The issue was that the old behavior could cause users to silently
overwrite data, which is pretty bad, so to be conservative we decided
to enforce the same checks that Hadoop does.

This was documented by this JIRA:
https://issues.apache.org/jira/browse/SPARK-1100
https://github.com/apache/spark/commit/3a8b698e961ac05d9d53e2bbf0c2844fcb1010d1

However, it would be very easy to add an option that allows preserving
the old behavior. Is anyone here interested in contributing that? I
created a JIRA for it:

https://issues.apache.org/jira/browse/SPARK-1993

- Patrick

On Mon, Jun 2, 2014 at 9:22 AM, Pierre Borckmans
 wrote:
> Indeed, the behavior has changed for good or for bad. I mean, I agree with
> the danger you mention but I'm not sure it's happening like that. Isn't
> there a mechanism for overwrite in Hadoop that automatically removes part
> files, then writes a _temporary folder and then only the part files along
> with the _success folder.
>
> In any case this change of behavior should be documented IMO.
>
> Cheers
> Pierre
>
> Message sent from a mobile device - excuse typos and abbreviations
>
> Le 2 juin 2014 à 17:42, Nicholas Chammas  a
> écrit :
>
> What I've found using saveAsTextFile() against S3 (prior to Spark 1.0.0.) is
> that files get overwritten automatically. This is one danger to this though.
> If I save to a directory that already has 20 part- files, but this time
> around I'm only saving 15 part- files, then there will be 5 leftover part-
> files from the previous set mixed in with the 15 newer files. This is
> potentially dangerous.
>
> I haven't checked to see if this behavior has changed in 1.0.0. Are you
> saying it has, Pierre?
>
> On Mon, Jun 2, 2014 at 9:41 AM, Pierre B
> [pierre.borckm...@realimpactanalytics.com](mailto:pierre.borckm...@realimpactanalytics.com)
> wrote:
>>
>> Hi Michaël,
>>
>> Thanks for this. We could indeed do that.
>>
>> But I guess the question is more about the change of behaviour from 0.9.1
>> to
>> 1.0.0.
>> We never had to care about that in previous versions.
>>
>> Does that mean we have to manually remove existing files or is there a way
>> to "aumotically" overwrite when using saveAsTextFile?
>>
>>
>>
>> --
>> View this message in context:
>> http://apache-spark-user-list.1001560.n3.nabble.com/How-can-I-make-Spark-1-0-saveAsTextFile-to-overwrite-existing-file-tp6696p6700.html
>> Sent from the Apache Spark User List mailing list archive at Nabble.com.


Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Pierre Borckmans
Indeed, the behavior has changed for good or for bad. I mean, I agree with the 
danger you mention but I'm not sure it's happening like that. Isn't there a 
mechanism for overwrite in Hadoop that automatically removes part files, then 
writes a _temporary folder and then only the part files along with the _success 
folder. 

In any case this change of behavior should be documented IMO.

Cheers 
Pierre

Message sent from a mobile device - excuse typos and abbreviations

> Le 2 juin 2014 à 17:42, Nicholas Chammas  a écrit 
> :
> 
> What I’ve found using saveAsTextFile() against S3 (prior to Spark 1.0.0.) is 
> that files get overwritten automatically. This is one danger to this though. 
> If I save to a directory that already has 20 part- files, but this time 
> around I’m only saving 15 part- files, then there will be 5 leftover part- 
> files from the previous set mixed in with the 15 newer files. This is 
> potentially dangerous.
> 
> I haven’t checked to see if this behavior has changed in 1.0.0. Are you 
> saying it has, Pierre?
> 
>> On Mon, Jun 2, 2014 at 9:41 AM, Pierre B 
>> [pierre.borckm...@realimpactanalytics.com](mailto:pierre.borckm...@realimpactanalytics.com)
>>  wrote:
>> 
>> Hi Michaël,
>> 
>> Thanks for this. We could indeed do that.
>> 
>> But I guess the question is more about the change of behaviour from 0.9.1 to
>> 1.0.0.
>> We never had to care about that in previous versions.
>> 
>> Does that mean we have to manually remove existing files or is there a way
>> to "aumotically" overwrite when using saveAsTextFile?
>> 
>> 
>> 
>> --
>> View this message in context: 
>> http://apache-spark-user-list.1001560.n3.nabble.com/How-can-I-make-Spark-1-0-saveAsTextFile-to-overwrite-existing-file-tp6696p6700.html
>> Sent from the Apache Spark User List mailing list archive at Nabble.com.
> 
> ​


Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Nicholas Chammas
What I’ve found using saveAsTextFile() against S3 (prior to Spark 1.0.0.)
is that files get overwritten automatically. This is one danger to this
though. If I save to a directory that already has 20 part- files, but this
time around I’m only saving 15 part- files, then there will be 5 leftover
part- files from the previous set mixed in with the 15 newer files. This is
potentially dangerous.

I haven’t checked to see if this behavior has changed in 1.0.0. Are you
saying it has, Pierre?

On Mon, Jun 2, 2014 at 9:41 AM, Pierre B
[pierre.borckm...@realimpactanalytics.com](mailto:pierre.borckm...@realimpactanalytics.com)

wrote:

Hi Michaël,
>
> Thanks for this. We could indeed do that.
>
> But I guess the question is more about the change of behaviour from 0.9.1
> to
> 1.0.0.
> We never had to care about that in previous versions.
>
> Does that mean we have to manually remove existing files or is there a way
> to "aumotically" overwrite when using saveAsTextFile?
>
>
>
> --
> View this message in context:
> http://apache-spark-user-list.1001560.n3.nabble.com/How-can-I-make-Spark-1-0-saveAsTextFile-to-overwrite-existing-file-tp6696p6700.html
> Sent from the Apache Spark User List mailing list archive at Nabble.com.
>
​


Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Pierre B
Hi Michaël,

Thanks for this. We could indeed do that.

But I guess the question is more about the change of behaviour from 0.9.1 to
1.0.0.
We never had to care about that in previous versions.

Does that mean we have to manually remove existing files or is there a way
to "aumotically" overwrite when using saveAsTextFile?



--
View this message in context: 
http://apache-spark-user-list.1001560.n3.nabble.com/How-can-I-make-Spark-1-0-saveAsTextFile-to-overwrite-existing-file-tp6696p6700.html
Sent from the Apache Spark User List mailing list archive at Nabble.com.


Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Michael Cutler
The function saveAsTextFile

is
a wrapper around saveAsHadoopFile

and
from looking at the source I don't see any flags etc. to overwrite existing
files.  It is however trivial to do this using HDFS directly from Scala.

val hadoopConf = new org.apache.hadoop.conf.Configuration()
val hdfs = org.apache.hadoop.fs.FileSystem.get(new
java.net.URI("hdfs://localhost:9000"), hadoopConf)


You can now use hdfs to do all sorts of useful things, listing directories,
recursively delete output directories e.g.

// Delete the existing path, ignore any exceptions thrown if the path
doesn't exist
val output = "hdfs://localhost:9000/tmp/wimbledon_top_mentions"
try { hdfs.delete(new org.apache.hadoop.fs.Path(output), true) } catch
{ case _ : Throwable => { } }
top_mentions.saveAsTextFile(output)


For an illustrated example of how I do this see HDFSDeleteExample.scala




*Michael Cutler*
Founder, CTO


*Mobile: +44 789 990 7847Email:   mich...@tumra.com Web:
tumra.com *
*Visit us at our offices in Chiswick Park *
*Registered in England & Wales, 07916412. VAT No. 130595328*


This email and any files transmitted with it are confidential and may also
be privileged. It is intended only for the person to whom it is addressed.
If you have received this email in error, please inform the sender immediately.
If you are not the intended recipient you must not use, disclose, copy,
print, distribute or rely on this email.


On 2 June 2014 09:26, Pierre Borckmans <
pierre.borckm...@realimpactanalytics.com> wrote:

> +1 Same question here...
>
> Message sent from a mobile device - excuse typos and abbreviations
>
> Le 2 juin 2014 à 10:08, Kexin Xie  a écrit :
>
> Hi,
>
> Spark 1.0 changes the default behaviour of RDD.saveAsTextFile to
> throw org.apache.hadoop.mapred.FileAlreadyExistsException when file already
> exists.
>
> Is there a way I can allow Spark to overwrite the existing file?
>
> Cheers,
> Kexin
>
>


Re: How can I make Spark 1.0 saveAsTextFile to overwrite existing file

2014-06-02 Thread Pierre Borckmans
+1 Same question here...

Message sent from a mobile device - excuse typos and abbreviations

> Le 2 juin 2014 à 10:08, Kexin Xie  a écrit :
> 
> Hi,
> 
> Spark 1.0 changes the default behaviour of RDD.saveAsTextFile to throw 
> org.apache.hadoop.mapred.FileAlreadyExistsException when file already exists. 
> 
> Is there a way I can allow Spark to overwrite the existing file?
> 
> Cheers,
> Kexin
>