spark-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Mridul Muralidharan (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (SPARK-21549) Spark fails to complete job correctly in case of OutputFormat which do not write into hdfs
Date Fri, 28 Jul 2017 22:17:00 GMT

    [ https://issues.apache.org/jira/browse/SPARK-21549?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16105630#comment-16105630
] 

Mridul Muralidharan edited comment on SPARK-21549 at 7/28/17 10:16 PM:
-----------------------------------------------------------------------

This affects both mapred ("mapred.output.dir") and mapreduce ("mapreduce.output.fileoutputformat.outputdir")
based OutputFormat's which do not set the properties referenced and is an incompatibility
introduced in spark 2.2

Workaround is to explicitly set the property to a dummy value (which is valid and writable
by user - say /tmp).

+CC [~WeiqingYang] 




was (Author: mridulm80):
This affects both mapred ("mapred.output.dir") and mapreduce ("mapreduce.output.fileoutputformat.outputdir")
based OutputFormat's which do not set the properties referenced and is an incompatibility
introduced in spark 2.2

Workaround is to explicitly set the property to a dummy value (which is valid and writable
by user).



> Spark fails to complete job correctly in case of OutputFormat which do not write into
hdfs
> ------------------------------------------------------------------------------------------
>
>                 Key: SPARK-21549
>                 URL: https://issues.apache.org/jira/browse/SPARK-21549
>             Project: Spark
>          Issue Type: Bug
>          Components: Spark Core
>    Affects Versions: 2.2.0
>         Environment: spark 2.2.0
> scala 2.11
>            Reporter: Sergey Zhemzhitsky
>
> Spark fails to complete job correctly in case of custom OutputFormat implementations.
> There are OutputFormat implementations which do not need to use *mapreduce.output.fileoutputformat.outputdir*
standard hadoop property.
> [But spark reads this property from the configuration|https://github.com/apache/spark/blob/v2.2.0/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopMapReduceWriter.scala#L79]
while setting up an OutputCommitter
> {code:javascript}
> val committer = FileCommitProtocol.instantiate(
>   className = classOf[HadoopMapReduceCommitProtocol].getName,
>   jobId = stageId.toString,
>   outputPath = conf.value.get("mapreduce.output.fileoutputformat.outputdir"),
>   isAppend = false).asInstanceOf[HadoopMapReduceCommitProtocol]
> committer.setupJob(jobContext)
> {code}
> ... and then uses this property later on while [commiting the job|https://github.com/apache/spark/blob/v2.2.0/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala#L132],
[aborting the job|https://github.com/apache/spark/blob/v2.2.0/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala#L141],
[creating task's temp path|https://github.com/apache/spark/blob/v2.2.0/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala#L95]
> In that cases when the job completes then following exception is thrown
> {code}
> Can not create a Path from a null string
> java.lang.IllegalArgumentException: Can not create a Path from a null string
>   at org.apache.hadoop.fs.Path.checkPathArg(Path.java:123)
>   at org.apache.hadoop.fs.Path.<init>(Path.java:135)
>   at org.apache.hadoop.fs.Path.<init>(Path.java:89)
>   at org.apache.spark.internal.io.HadoopMapReduceCommitProtocol.absPathStagingDir(HadoopMapReduceCommitProtocol.scala:58)
>   at org.apache.spark.internal.io.HadoopMapReduceCommitProtocol.abortJob(HadoopMapReduceCommitProtocol.scala:141)
>   at org.apache.spark.internal.io.SparkHadoopMapReduceWriter$.write(SparkHadoopMapReduceWriter.scala:106)
>   at org.apache.spark.rdd.PairRDDFunctions$$anonfun$saveAsNewAPIHadoopDataset$1.apply$mcV$sp(PairRDDFunctions.scala:1085)
>   at org.apache.spark.rdd.PairRDDFunctions$$anonfun$saveAsNewAPIHadoopDataset$1.apply(PairRDDFunctions.scala:1085)
>   at org.apache.spark.rdd.PairRDDFunctions$$anonfun$saveAsNewAPIHadoopDataset$1.apply(PairRDDFunctions.scala:1085)
>   at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
>   at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
>   at org.apache.spark.rdd.RDD.withScope(RDD.scala:362)
>   at org.apache.spark.rdd.PairRDDFunctions.saveAsNewAPIHadoopDataset(PairRDDFunctions.scala:1084)
>   ...
> {code}
> So it seems that all the jobs which use OutputFormats which don't write data into HDFS-compatible
file systems are broken.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org


Mime
View raw message