You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Romain Giot (JIRA)" <ji...@apache.org> on 2016/06/21 07:18:57 UTC

[jira] [Created] (SPARK-16091) Dataset.partitionBy.csv raise a java.io.FileNotFoundException when launched on an hadoop cluster

Romain Giot created SPARK-16091:
-----------------------------------

             Summary: Dataset.partitionBy.csv raise a java.io.FileNotFoundException when launched on an hadoop cluster
                 Key: SPARK-16091
                 URL: https://issues.apache.org/jira/browse/SPARK-16091
             Project: Spark
          Issue Type: Bug
          Components: Input/Output
    Affects Versions: 2.0.0
         Environment: Hadoop version: 2.5.1 
            Reporter: Romain Giot
            Priority: Blocker


When writing a `Dataset` in a `CSV` file, the following exception  java.io.FileNotFoundException is raised *after* the writing is done and successful.

This behaviour does not happen when the spark application is launched locally ; it should be related to `hdfs` management.

Here is a test code:
```scala
import org.apache.spark.SparkContext
import org.apache.hadoop.fs.FileSystem
import org.apache.hadoop.fs.{Path, PathFilter}
import org.apache.spark.sql.SQLContext



case class Test(A: String, B: String, C:String){
}

object WriteTest {
  val sc: SparkContext = new SparkContext()
  val fs: FileSystem = FileSystem.get(sc.hadoopConfiguration)
  val sqlContext: SQLContext = new SQLContext(sc)
  
  import sqlContext.implicits._

  def main(args: Array[String]):Unit = {

    val ds = Seq(
      Test("abc", "abc", "abc"),
      Test("abc", "abc", "def"), 
      Test("abc", "abc", "ghi"), 
      Test("abc", "xyz", "abc"),
      Test("xyz", "xyz", "abc")
    ).toDS()


    // works
    ds
      .write
      .option("header",true)
      .mode("overwrite")
      .csv("/tmp/test1.csv")

    // fails
    ds
      .write
      .option("header",true)
      .mode("overwrite")
      .partitionBy("A", "B")
      .csv("/tmp/test2.csv")

  }
}
```

and here is the exception stack:
```
16/06/21 08:59:33 ERROR yarn.ApplicationMaster: User class threw exception: java.io.FileNotFoundException: Path is not a file: /tmp/test2.csv/A=abc
	at org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:68)
	at org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:54)
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocationsUpdateTimes(FSNamesystem.java:1795)
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocationsInt(FSNamesystem.java:1738)
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1718)
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1690)
	at org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getBlockLocations(NameNodeRpcServer.java:519)
	at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getBlockLocations(ClientNamenodeProtocolServerSideTranslatorPB.java:337)
	at org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
	at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:585)
	at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:928)
	at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2013)
	at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2009)
	at java.security.AccessController.doPrivileged(Native Method)
	at javax.security.auth.Subject.doAs(Subject.java:422)
	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1614)
	at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2007)
```

I have no idea if the bug comes from the `hdfs` implementation or the way `spark` uses it.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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