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 <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 <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 <
> 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 <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 <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 W: www.velos.io
>
>
>


-- 
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

Reply via email to