[jira] [Commented] (SPARK-21549) Spark fails to complete job correctly in case of OutputFormat which do not write into hdfs
[ https://issues.apache.org/jira/browse/SPARK-21549?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16204504#comment-16204504 ] Apache Spark commented on SPARK-21549: -- User 'mridulm' has created a pull request for this issue: https://github.com/apache/spark/pull/19497 > 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 > Fix For: 2.2.1, 2.3.0 > > > 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.(Path.java:135) > at org.apache.hadoop.fs.Path.(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-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-21549) Spark fails to complete job correctly in case of OutputFormat which do not write into hdfs
[ https://issues.apache.org/jira/browse/SPARK-21549?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16202865#comment-16202865 ] Apache Spark commented on SPARK-21549: -- User 'mridulm' has created a pull request for this issue: https://github.com/apache/spark/pull/19487 > 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 > Fix For: 2.2.1, 2.3.0 > > > 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.(Path.java:135) > at org.apache.hadoop.fs.Path.(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-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-21549) Spark fails to complete job correctly in case of OutputFormat which do not write into hdfs
[ https://issues.apache.org/jira/browse/SPARK-21549?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16173755#comment-16173755 ] Sergey Zhemzhitsky commented on SPARK-21549: [~ste...@apache.org] I've updated PR to prevent using FileSystems at all. Instead, there is just an additional check whether there are absolute files to rename during commit. > 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.(Path.java:135) > at org.apache.hadoop.fs.Path.(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-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-21549) Spark fails to complete job correctly in case of OutputFormat which do not write into hdfs
[ https://issues.apache.org/jira/browse/SPARK-21549?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16173449#comment-16173449 ] Steve Loughran commented on SPARK-21549: The {{newTaskTempFileAbsPath()}} method is an interesting spot of code...I'm still trying to work out when it is actually used. Some committers like {{ManifestFileCommitProtocol}} don't support it all. However, if it is used, then your patch is going to cause problems if the dest FS != the default FS, because then the bit of the protocol which takes that list of temp files and renames() them into their destination is going to fail. I think you'd be better off having the committer fail fast when an absolute path is asked for > 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.(Path.java:135) > at org.apache.hadoop.fs.Path.(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-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-21549) Spark fails to complete job correctly in case of OutputFormat which do not write into hdfs
[ https://issues.apache.org/jira/browse/SPARK-21549?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16173184#comment-16173184 ] Sergey Zhemzhitsky commented on SPARK-21549: [~mridulm80], [~WeiqingYang], [~ste...@apache.org], I've implemented the fix in this PR (https://github.com/apache/spark/pull/19294), which sets user's current working directory (which is typically her home directory in case of distributed filesystems) as output directory. The patch allows using OutputFormats which write to external systems, databases, etc. by means of RDD API. I far as I understand the requirement for output paths to be specified is only necessary to allow files to be committed to an absolute output location, that is not the case for output formats which write data to external systems. So using user's working directory for such situations seems to be ok. > 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.(Path.java:135) > at org.apache.hadoop.fs.Path.(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-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-21549) Spark fails to complete job correctly in case of OutputFormat which do not write into hdfs
[ https://issues.apache.org/jira/browse/SPARK-21549?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16173175#comment-16173175 ] Apache Spark commented on SPARK-21549: -- User 'szhem' has created a pull request for this issue: https://github.com/apache/spark/pull/19294 > 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.(Path.java:135) > at org.apache.hadoop.fs.Path.(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-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-21549) Spark fails to complete job correctly in case of OutputFormat which do not write into hdfs
[ https://issues.apache.org/jira/browse/SPARK-21549?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16171420#comment-16171420 ] Steve Loughran commented on SPARK-21549: # you can't rely on the committers having output and temp dirs. Subclasses of {{FileOutputCommitter}} *must*, though there's no official mechanism for querying that because {{getOutputPath()}} is private. # Hadoop 3.0 has added (MAPREDUCE-6956) a new superclass of {{FileOutputCommitter}}, [PathOutputCommitter|[https://github.com/apache/hadoop/blob/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/PathOutputCommitter.java] which pulls up getWorkingDir method to be more general (so that you can have output committers which tell spark and hive where their intermediate data should go, without them being subclasses of FileOutputCommitter. # I'm happy to pull up {{getOutputPath}} to that class, and if people can give me a patch for it *this week* I'll add it for 3.0 beta 1. Regarding the committer here, you might want to think of moving off/subclassing {{HadoopMapReduceCommitProtocol}}. This is what I've done in [PathOutputCommitProtoco|https://github.com/hortonworks-spark/cloud-integration/blob/master/spark-cloud-integration/src/main/scala/com/hortonworks/spark/cloud/PathOutputCommitProtocol.scala], though I can see it's still relying on the superclass to get that properly. Again, if we can patch the new PathOutputCommitter class for a getOutputPath I use that here. And yes, while that new mapreduce will take a long time to surface in spark core, you can use it independently, from later this year.. If you are playing with different committers out of spark's own codebase, pick up the the ORC hive tests from [https://github.com/hortonworks-spark/cloud-integration/tree/master/cloud-examples/src/test/scala/org/apache/spark/sql/sources]. These are just some of the spark sql tests reworked slightly so that they'll work with any FileSystem impl. rather than just local file:// paths Ping me direct if you are playing with new committers, & look at MAPREDUCE-6823 to see if that'd be useful to you (& how it could be improved, given its still not in the codebase) > 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.(Path.java:135) > at org.apache.hadoop.fs.Path.(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 >
[jira] [Commented] (SPARK-21549) Spark fails to complete job correctly in case of OutputFormat which do not write into hdfs
[ https://issues.apache.org/jira/browse/SPARK-21549?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16171393#comment-16171393 ] Steve Loughran commented on SPARK-21549: Linking to SPARK-20045, which highlights the commit logic, especially the abort code, needs to be resilient to failures, including that of the invoked {{OutputCommitter.abort()}} calls from raising exceptions. While people implementing committers should be expected to write resilient abort routines, you can't rely on it. Same for calling fs.delete()...it could also fall, so wrapping everything in an exception handler would at least make abort resilient. > 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.(Path.java:135) > at org.apache.hadoop.fs.Path.(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-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-21549) Spark fails to complete job correctly in case of OutputFormat which do not write into hdfs
[ https://issues.apache.org/jira/browse/SPARK-21549?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16135545#comment-16135545 ] Sergey Zhemzhitsky commented on SPARK-21549: [~mridulm80], [~WeiqingYang] does it make sense to implement the provided workaround with valid and writable directory just within SparkHadoopMapRedceWriter if the needed property is not set, to prevent affecting all the jobs which don't write to hdfs, at least until there is a better solution? > 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.(Path.java:135) > at org.apache.hadoop.fs.Path.(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-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-21549) Spark fails to complete job correctly in case of OutputFormat which do not write into hdfs
[ https://issues.apache.org/jira/browse/SPARK-21549?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16105630#comment-16105630 ] Mridul Muralidharan commented on SPARK-21549: - 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. > 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.(Path.java:135) > at org.apache.hadoop.fs.Path.(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-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org