Re: DirectFileOutputCommiter

2016-02-29 Thread Steve Loughran
> On 26 Feb 2016, at 06:24, Takeshi Yamamuro wrote: > > Hi, > > Great work! > What is the concrete performance gain of the committer on s3? > I'd like to know. > > I think there is no direct committer for files because these kinds of > committer has risks > to loss data (See: SPARK-10063). >

Re: DirectFileOutputCommiter

2016-02-29 Thread Takeshi Yamamuro
Hi, I think the essential culprit is that these committers are not idempotent; retry attempts will fail. See codes below for details; https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala#L130 On Sat, Feb 27, 2016 at 7

Re: DirectFileOutputCommiter

2016-02-27 Thread Igor Berman
Hi Reynold, thanks for the response Yes, speculation mode needs some coordination. Regarding job failure : correct me if I wrong - if one of jobs fails - client code will be sort of "notified" by exception or something similar, so the client can decide to re-submit action(job), i.e. it won't be "si

Re: DirectFileOutputCommiter

2016-02-26 Thread Alexander Pivovarov
DirectOutputCommitter doc says: The FileOutputCommitter is required for HDFS + speculation, which allows only one writer at a time for a file (so two people racing to write the same file would not work). However, S3 supports multiple writers outputting to the same file, where visibility is guaran

Re: DirectFileOutputCommiter

2016-02-26 Thread Reynold Xin
It could lose data in speculation mode, or if any job fails. On Fri, Feb 26, 2016 at 3:45 AM, Igor Berman wrote: > Takeshi, do you know the reason why they wanted to remove this commiter in > SPARK-10063? > the jira has no info inside > as far as I understand the direct committer can't be used w

Re: DirectFileOutputCommiter

2016-02-26 Thread Igor Berman
Takeshi, do you know the reason why they wanted to remove this commiter in SPARK-10063? the jira has no info inside as far as I understand the direct committer can't be used when either of two is true 1. speculation mode 2. append mode(ie. not creating new version of data but appending to existing

Re: DirectFileOutputCommiter

2016-02-26 Thread Igor Berman
Alexander, implementation you've attaches supports both modes configured by property " mapred.output.direct." + fs.getClass().getSimpleName() as soon as you see _temporary dir probably the mode is off i.e. the default impl is working and you experiencing some other problem. On 26 February 2016 at

Re: DirectFileOutputCommiter

2016-02-26 Thread Igor Berman
the performance gain is for commit stage when data is moved from _temporary directory to distination directory since s3 is key-value really the move operation is like copy operation On 26 February 2016 at 08:24, Takeshi Yamamuro wrote: > Hi, > > Great work! > What is the concrete performance ga

Re: DirectFileOutputCommiter

2016-02-26 Thread Teng Qiu
Hi, thanks :) performance gain is huge, we have a INSERT INTO query, ca. 30GB in JSON format will be written to s3 at the end, without DirectOutputCommitter and our hack in hive and InsertIntoHiveTable.scala, it took more than 40min, with our changes, only 15min then. DirectOutputCommitter works f

Re: DirectFileOutputCommiter

2016-02-26 Thread Alexander Pivovarov
Amazon uses the following impl https://gist.github.com/apivovarov/bb215f08318318570567 But for some reason Spark show error at the end of the job 16/02/26 08:16:54 INFO scheduler.DAGScheduler: ResultStage 0 (saveAsTextFile at :28) finished in 14.305 s 16/02/26 08:16:54 INFO cluster.YarnScheduler:

Re: DirectFileOutputCommiter

2016-02-25 Thread Takeshi Yamamuro
Hi, Great work! What is the concrete performance gain of the committer on s3? I'd like to know. I think there is no direct committer for files because these kinds of committer has risks to loss data (See: SPARK-10063). Until this resolved, ISTM files cannot support direct commits. thanks, On

Re: DirectFileOutputCommiter

2016-02-25 Thread Teng Qiu
yes, should be this one https://gist.github.com/aarondav/c513916e72101bbe14ec then need to set it in spark-defaults.conf : https://github.com/zalando/spark/commit/3473f3f1ef27830813c1e0b3686e96a55f49269c#diff-f7a46208be9e80252614369be6617d65R13 Am Freitag, 26. Februar 2016 schrieb Yin Yang : > Th

Re: DirectFileOutputCommiter

2016-02-25 Thread Yin Yang
The header of DirectOutputCommitter.scala says Databricks. Did you get it from Databricks ? On Thu, Feb 25, 2016 at 3:01 PM, Teng Qiu wrote: > interesting in this topic as well, why the DirectFileOutputCommitter not > included? > > we added it in our fork, under > core/src/main/scala/org/apache

Re: DirectFileOutputCommiter

2016-02-25 Thread Teng Qiu
interesting in this topic as well, why the DirectFileOutputCommitter not included? we added it in our fork, under core/src/main/scala/org/apache/spark/mapred/DirectOutputCommitter.scala moreover, this DirectFileOutputCommitter is not working for the insert operations in HiveContext, since the Com