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 10:57, Alexander Pivovarov <apivova...@gmail.com> wrote: > 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 <console>:28) finished in 14.305 s > 16/02/26 08:16:54 INFO cluster.YarnScheduler: Removed TaskSet 0.0, whose > tasks have all completed, from pool > 16/02/26 08:16:54 INFO scheduler.DAGScheduler: Job 0 finished: > saveAsTextFile at <console>:28, took 14.467271 s > java.io.FileNotFoundException: File > s3n://my-backup/test/test1/_temporary/0 does not exist. > at > org.apache.hadoop.fs.s3native.NativeS3FileSystem.listStatus(NativeS3FileSystem.java:564) > at org.apache.hadoop.fs.FileSystem.listStatus(FileSystem.java:1485) > at org.apache.hadoop.fs.FileSystem.listStatus(FileSystem.java:1525) > at > org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.getAllCommittedTaskPaths(FileOutputCommitter.java:269) > at > org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.commitJob(FileOutputCommitter.java:309) > at > org.apache.hadoop.mapred.FileOutputCommitter.commitJob(FileOutputCommitter.java:136) > at > org.apache.spark.SparkHadoopWriter.commitJob(SparkHadoopWriter.scala:112) > at > org.apache.spark.rdd.PairRDDFunctions$$anonfun$saveAsHadoopDataset$1.apply$mcV$sp(PairRDDFunctions.scala:1214) > at > org.apache.spark.rdd.PairRDDFunctions$$anonfun$saveAsHadoopDataset$1.apply(PairRDDFunctions.scala:1156) > at > org.apache.spark.rdd.PairRDDFunctions$$anonfun$saveAsHadoopDataset$1.apply(PairRDDFunctions.scala:1156) > at > org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:150) > at > org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:111) > at org.apache.spark.rdd.RDD.withScope(RDD.scala:316) > > > Another implementation works fine > https://gist.github.com/aarondav/c513916e72101bbe14ec > > On Thu, Feb 25, 2016 at 10:24 PM, Takeshi Yamamuro <linguin....@gmail.com> > 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). >> Until this resolved, ISTM files cannot support direct commits. >> >> thanks, >> >> >> >> On Fri, Feb 26, 2016 at 8:39 AM, Teng Qiu <teng...@gmail.com> wrote: >> >>> 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 : >>> > The header of DirectOutputCommitter.scala says Databricks. >>> > Did you get it from Databricks ? >>> > On Thu, Feb 25, 2016 at 3:01 PM, Teng Qiu <teng...@gmail.com> 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/spark/mapred/DirectOutputCommitter.scala >>> >> moreover, this DirectFileOutputCommitter is not working for the >>> insert operations in HiveContext, since the Committer is called by hive >>> (means uses dependencies in hive package) >>> >> we made some hack to fix this, you can take a look: >>> >> >>> https://github.com/apache/spark/compare/branch-1.6...zalando:branch-1.6-zalando >>> >> >>> >> may bring some ideas to other spark contributors to find a better way >>> to use s3. >>> >> >>> >> 2016-02-22 23:18 GMT+01:00 igor.berman <igor.ber...@gmail.com>: >>> >>> >>> >>> Hi, >>> >>> Wanted to understand if anybody uses DirectFileOutputCommitter or >>> alikes >>> >>> especially when working with s3? >>> >>> I know that there is one impl in spark distro for parquet format, >>> but not >>> >>> for files - why? >>> >>> >>> >>> Imho, it can bring huge performance boost. >>> >>> Using default FileOutputCommiter with s3 has big overhead at commit >>> stage >>> >>> when all parts are copied one-by-one to destination dir from >>> _temporary, >>> >>> which is bottleneck when number of partitions is high. >>> >>> >>> >>> Also, wanted to know if there are some problems when using >>> >>> DirectFileOutputCommitter? >>> >>> If writing one partition directly will fail in the middle is spark >>> will >>> >>> notice this and will fail job(say after all retries)? >>> >>> >>> >>> thanks in advance >>> >>> >>> >>> >>> >>> >>> >>> >>> >>> -- >>> >>> View this message in context: >>> http://apache-spark-user-list.1001560.n3.nabble.com/DirectFileOutputCommiter-tp26296.html >>> >>> Sent from the Apache Spark User List mailing list archive at >>> Nabble.com. >>> >>> >>> >>> --------------------------------------------------------------------- >>> >>> To unsubscribe, e-mail: user-unsubscr...@spark.apache.org >>> >>> For additional commands, e-mail: user-h...@spark.apache.org >>> >>> >>> >> >>> > >>> > >>> >> >> >> >> -- >> --- >> Takeshi Yamamuro >> > >