Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-21549

Spark fails to complete job correctly in case of OutputFormat which do not write into hdfs

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 2.2.0
    • 2.2.1, 2.3.0
    • Spark Core
    • None
    • spark 2.2.0
      scala 2.11

    Description

      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 while setting up an OutputCommitter

      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)
      

      ... and then uses this property later on while commiting the job, aborting the job, creating task's temp path

      In that cases when the job completes then following exception is thrown

      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)
        ...
      

      So it seems that all the jobs which use OutputFormats which don't write data into HDFS-compatible file systems are broken.

      Attachments

        Issue Links

          Activity

            People

              szhemzhitsky Sergey Zhemzhitsky
              szhemzhitsky Sergey Zhemzhitsky
              Votes:
              2 Vote for this issue
              Watchers:
              10 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: