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 <nicholas.chammas@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<ilikerps@gmail.com>님이 작성한 메시지:
Yes.


On Mon, Jun 2, 2014 at 1:23 PM, Nicholas Chammas <nicholas.chammas@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 <ilikerps@gmail.com> wrote:
+1 please re-add this feature


On Mon, Jun 2, 2014 at 12:44 PM, Patrick Wendell <pwendell@gmail.com> 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 <zhunanmcgill@gmail.com> 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
> <pierre.borckmans@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 <nicholas.chammas@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