[jira] [Commented] (SPARK-14959) ​Problem Reading partitioned ORC or Parquet files

2017-11-17 Thread Steve Loughran (JIRA)

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

Steve Loughran commented on SPARK-14959:


Came across a reference to this while scanning for getFileBlockLocations() use.

HDFS shouldn't be throwing this. {{getFileBlockLocations(Path, offset, len)}} 
is nominally the same as {{getFileBlockLocations(getFileStatus(Path), offset, 
len)}}; the latter will return an empty array on a directory. Looks like the 
HDFS behaviour has been there for years, and people can argue that it's the 
correct behaviour: but its the only subclass of the base FileSystem 
implementation, and it doesn't fail on a directory. Maybe it can be fixed, at 
the very least the behaviour needs to be specified explicitly. 

> ​Problem Reading partitioned ORC or Parquet files
> -
>
> Key: SPARK-14959
> URL: https://issues.apache.org/jira/browse/SPARK-14959
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.0.0
> Environment: Hadoop 2.7.1.2.4.0.0-169 (HDP 2.4)
>Reporter: Sebastian YEPES FERNANDEZ
>Assignee: Xin Wu
>Priority: Blocker
> Fix For: 2.0.0
>
>
> Hello,
> I have noticed that in the pasts days there is an issue when trying to read 
> partitioned files from HDFS.
> I am running on Spark master branch #c544356
> The write actually works but the read fails.
> {code:title=Issue Reproduction}
> case class Data(id: Int, text: String)
> val ds = spark.createDataset( Seq(Data(0, "hello"), Data(1, "hello"), Data(0, 
> "world"), Data(1, "there")) )
> scala> 
> ds.write.mode(org.apache.spark.sql.SaveMode.Overwrite).format("parquet").partitionBy("id").save("/user/spark/test.parquet")
> SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".  
>   
> SLF4J: Defaulting to no-operation (NOP) logger implementation
> SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further 
> details.
> java.io.FileNotFoundException: Path is not a file: 
> /user/spark/test.parquet/id=0
> at 
> org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:75)
> at 
> org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:61)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocationsInt(FSNamesystem.java:1828)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1799)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1712)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getBlockLocations(NameNodeRpcServer.java:652)
> at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getBlockLocations(ClientNamenodeProtocolServerSideTranslatorPB.java:365)
> at 
> org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:616)
> at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:969)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2151)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2147)
> 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:1657)
> at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2145)
>   at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
>   at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
>   at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
>   at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
>   at 
> org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:106)
>   at 
> org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:73)
>   at 
> org.apache.hadoop.hdfs.DFSClient.callGetBlockLocations(DFSClient.java:1242)
>   at org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1227)
>   at org.apache.hadoop.hdfs.DFSClient.getBlockLocations(DFSClient.java:1285)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$1.doCall(DistributedFileSystem.java:221)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$1.doCall(DistributedFileSystem.java:217)
>   at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>   at 
> 

[jira] [Commented] (SPARK-14959) ​Problem Reading partitioned ORC or Parquet files

2016-06-01 Thread Apache Spark (JIRA)

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

Apache Spark commented on SPARK-14959:
--

User 'xwu0226' has created a pull request for this issue:
https://github.com/apache/spark/pull/13463

> ​Problem Reading partitioned ORC or Parquet files
> -
>
> Key: SPARK-14959
> URL: https://issues.apache.org/jira/browse/SPARK-14959
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.0.0
> Environment: Hadoop 2.7.1.2.4.0.0-169 (HDP 2.4)
>Reporter: Sebastian YEPES FERNANDEZ
>Priority: Blocker
>
> Hello,
> I have noticed that in the pasts days there is an issue when trying to read 
> partitioned files from HDFS.
> I am running on Spark master branch #c544356
> The write actually works but the read fails.
> {code:title=Issue Reproduction}
> case class Data(id: Int, text: String)
> val ds = spark.createDataset( Seq(Data(0, "hello"), Data(1, "hello"), Data(0, 
> "world"), Data(1, "there")) )
> scala> 
> ds.write.mode(org.apache.spark.sql.SaveMode.Overwrite).format("parquet").partitionBy("id").save("/user/spark/test.parquet")
> SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".  
>   
> SLF4J: Defaulting to no-operation (NOP) logger implementation
> SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further 
> details.
> java.io.FileNotFoundException: Path is not a file: 
> /user/spark/test.parquet/id=0
> at 
> org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:75)
> at 
> org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:61)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocationsInt(FSNamesystem.java:1828)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1799)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1712)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getBlockLocations(NameNodeRpcServer.java:652)
> at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getBlockLocations(ClientNamenodeProtocolServerSideTranslatorPB.java:365)
> at 
> org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:616)
> at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:969)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2151)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2147)
> 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:1657)
> at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2145)
>   at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
>   at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
>   at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
>   at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
>   at 
> org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:106)
>   at 
> org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:73)
>   at 
> org.apache.hadoop.hdfs.DFSClient.callGetBlockLocations(DFSClient.java:1242)
>   at org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1227)
>   at org.apache.hadoop.hdfs.DFSClient.getBlockLocations(DFSClient.java:1285)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$1.doCall(DistributedFileSystem.java:221)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$1.doCall(DistributedFileSystem.java:217)
>   at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileBlockLocations(DistributedFileSystem.java:228)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileBlockLocations(DistributedFileSystem.java:209)
>   at 
> org.apache.spark.sql.execution.datasources.HDFSFileCatalog$$anonfun$9$$anonfun$apply$4.apply(fileSourceInterfaces.scala:372)
>   at 
> org.apache.spark.sql.execution.datasources.HDFSFileCatalog$$anonfun$9$$anonfun$apply$4.apply(fileSourceInterfaces.scala:360)
>   at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>   at 
> 

[jira] [Commented] (SPARK-14959) ​Problem Reading partitioned ORC or Parquet files

2016-06-01 Thread Xin Wu (JIRA)

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

Xin Wu commented on SPARK-14959:


I can recreate the problem with hdfs location. and I have a patch for it now. I 
will submit a PR soon. 

The actual results now is following, as expected:
{code}
scala> 
spark.read.format("parquet").load("hdfs://bdavm009.svl.ibm.com:8020/user/spark/SPARK-14959_part").show
+-+---+
| text| id|
+-+---+
|hello|  0|
|world|  0|
|hello|  1|
|there|  1|
+-+---+

   
spark.read.format("orc").load("hdfs://bdavm009.svl.ibm.com:8020/user/spark/SPARK-14959_orc").show
+-+---+
| text| id|
+-+---+
|hello|  0|
|world|  0|
|hello|  1|
|there|  1|
+-+---+
{code}

> ​Problem Reading partitioned ORC or Parquet files
> -
>
> Key: SPARK-14959
> URL: https://issues.apache.org/jira/browse/SPARK-14959
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.0.0
> Environment: Hadoop 2.7.1.2.4.0.0-169 (HDP 2.4)
>Reporter: Sebastian YEPES FERNANDEZ
>Priority: Blocker
>
> Hello,
> I have noticed that in the pasts days there is an issue when trying to read 
> partitioned files from HDFS.
> I am running on Spark master branch #c544356
> The write actually works but the read fails.
> {code:title=Issue Reproduction}
> case class Data(id: Int, text: String)
> val ds = spark.createDataset( Seq(Data(0, "hello"), Data(1, "hello"), Data(0, 
> "world"), Data(1, "there")) )
> scala> 
> ds.write.mode(org.apache.spark.sql.SaveMode.Overwrite).format("parquet").partitionBy("id").save("/user/spark/test.parquet")
> SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".  
>   
> SLF4J: Defaulting to no-operation (NOP) logger implementation
> SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further 
> details.
> java.io.FileNotFoundException: Path is not a file: 
> /user/spark/test.parquet/id=0
> at 
> org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:75)
> at 
> org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:61)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocationsInt(FSNamesystem.java:1828)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1799)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1712)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getBlockLocations(NameNodeRpcServer.java:652)
> at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getBlockLocations(ClientNamenodeProtocolServerSideTranslatorPB.java:365)
> at 
> org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:616)
> at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:969)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2151)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2147)
> 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:1657)
> at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2145)
>   at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
>   at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
>   at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
>   at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
>   at 
> org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:106)
>   at 
> org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:73)
>   at 
> org.apache.hadoop.hdfs.DFSClient.callGetBlockLocations(DFSClient.java:1242)
>   at org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1227)
>   at org.apache.hadoop.hdfs.DFSClient.getBlockLocations(DFSClient.java:1285)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$1.doCall(DistributedFileSystem.java:221)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$1.doCall(DistributedFileSystem.java:217)
>   at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileBlockLocations(DistributedFileSystem.java:228)
>   at 
> 

[jira] [Commented] (SPARK-14959) ​Problem Reading partitioned ORC or Parquet files

2016-05-17 Thread Jurriaan Pruis (JIRA)

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

Jurriaan Pruis commented on SPARK-14959:


As you can see in the description writing is also broken as it throws an 
exception. It doesn't write the schema properly 
(https://github.com/apache/spark/blob/bc3760d405cc8c3ffcd957b188afa8b7e3b1f824/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala#L431)
 because of the `resolveRelation` eventually hitting the same kind of 
`FileNotFoundException`.

> ​Problem Reading partitioned ORC or Parquet files
> -
>
> Key: SPARK-14959
> URL: https://issues.apache.org/jira/browse/SPARK-14959
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.0.0
> Environment: Hadoop 2.7.1.2.4.0.0-169 (HDP 2.4)
>Reporter: Sebastian YEPES FERNANDEZ
>
> Hello,
> I have noticed that in the pasts days there is an issue when trying to read 
> partitioned files from HDFS.
> I am running on Spark master branch #c544356
> The write actually works but the read fails.
> {code:title=Issue Reproduction}
> case class Data(id: Int, text: String)
> val ds = spark.createDataset( Seq(Data(0, "hello"), Data(1, "hello"), Data(0, 
> "world"), Data(1, "there")) )
> scala> 
> ds.write.mode(org.apache.spark.sql.SaveMode.Overwrite).format("parquet").partitionBy("id").save("/user/spark/test.parquet")
> SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".  
>   
> SLF4J: Defaulting to no-operation (NOP) logger implementation
> SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further 
> details.
> java.io.FileNotFoundException: Path is not a file: 
> /user/spark/test.parquet/id=0
> at 
> org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:75)
> at 
> org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:61)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocationsInt(FSNamesystem.java:1828)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1799)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1712)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getBlockLocations(NameNodeRpcServer.java:652)
> at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getBlockLocations(ClientNamenodeProtocolServerSideTranslatorPB.java:365)
> at 
> org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:616)
> at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:969)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2151)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2147)
> 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:1657)
> at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2145)
>   at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
>   at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
>   at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
>   at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
>   at 
> org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:106)
>   at 
> org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:73)
>   at 
> org.apache.hadoop.hdfs.DFSClient.callGetBlockLocations(DFSClient.java:1242)
>   at org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1227)
>   at org.apache.hadoop.hdfs.DFSClient.getBlockLocations(DFSClient.java:1285)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$1.doCall(DistributedFileSystem.java:221)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$1.doCall(DistributedFileSystem.java:217)
>   at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileBlockLocations(DistributedFileSystem.java:228)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileBlockLocations(DistributedFileSystem.java:209)
>   at 
> org.apache.spark.sql.execution.datasources.HDFSFileCatalog$$anonfun$9$$anonfun$apply$4.apply(fileSourceInterfaces.scala:372)
>   at 
> 

[jira] [Commented] (SPARK-14959) ​Problem Reading partitioned ORC or Parquet files

2016-05-12 Thread Sebastian YEPES FERNANDEZ (JIRA)

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

Sebastian YEPES FERNANDEZ commented on SPARK-14959:
---

I think this issue was introduced around SPARK-13664, but the thing is that 
there have been many underlining changes.
If you need anymore debugging info let me know. 

> ​Problem Reading partitioned ORC or Parquet files
> -
>
> Key: SPARK-14959
> URL: https://issues.apache.org/jira/browse/SPARK-14959
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.0.0
> Environment: Hadoop 2.7.1.2.4.0.0-169 (HDP 2.4)
>Reporter: Sebastian YEPES FERNANDEZ
>
> Hello,
> I have noticed that in the pasts days there is an issue when trying to read 
> partitioned files from HDFS.
> I am running on Spark master branch #c544356
> The write actually works but the read fails.
> {code:title=Issue Reproduction}
> case class Data(id: Int, text: String)
> val ds = spark.createDataset( Seq(Data(0, "hello"), Data(1, "hello"), Data(0, 
> "world"), Data(1, "there")) )
> scala> 
> ds.write.mode(org.apache.spark.sql.SaveMode.Overwrite).format("parquet").partitionBy("id").save("/user/spark/test.parquet")
> SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".  
>   
> SLF4J: Defaulting to no-operation (NOP) logger implementation
> SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further 
> details.
> java.io.FileNotFoundException: Path is not a file: 
> /user/spark/test.parquet/id=0
> at 
> org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:75)
> at 
> org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:61)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocationsInt(FSNamesystem.java:1828)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1799)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1712)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getBlockLocations(NameNodeRpcServer.java:652)
> at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getBlockLocations(ClientNamenodeProtocolServerSideTranslatorPB.java:365)
> at 
> org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:616)
> at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:969)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2151)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2147)
> 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:1657)
> at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2145)
>   at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
>   at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
>   at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
>   at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
>   at 
> org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:106)
>   at 
> org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:73)
>   at 
> org.apache.hadoop.hdfs.DFSClient.callGetBlockLocations(DFSClient.java:1242)
>   at org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1227)
>   at org.apache.hadoop.hdfs.DFSClient.getBlockLocations(DFSClient.java:1285)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$1.doCall(DistributedFileSystem.java:221)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$1.doCall(DistributedFileSystem.java:217)
>   at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileBlockLocations(DistributedFileSystem.java:228)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileBlockLocations(DistributedFileSystem.java:209)
>   at 
> org.apache.spark.sql.execution.datasources.HDFSFileCatalog$$anonfun$9$$anonfun$apply$4.apply(fileSourceInterfaces.scala:372)
>   at 
> org.apache.spark.sql.execution.datasources.HDFSFileCatalog$$anonfun$9$$anonfun$apply$4.apply(fileSourceInterfaces.scala:360)
>   at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>   at 

[jira] [Commented] (SPARK-14959) ​Problem Reading partitioned ORC or Parquet files

2016-05-11 Thread Jurriaan Pruis (JIRA)

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

Jurriaan Pruis commented on SPARK-14959:


I have the same issue reading a partitioned parquet table using Spark 2.0.0 
(which was saved originally using Spark 1.6.1)

> ​Problem Reading partitioned ORC or Parquet files
> -
>
> Key: SPARK-14959
> URL: https://issues.apache.org/jira/browse/SPARK-14959
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.0.0
> Environment: Hadoop 2.7.1.2.4.0.0-169 (HDP 2.4)
>Reporter: Sebastian YEPES FERNANDEZ
>
> Hello,
> I have noticed that in the pasts days there is an issue when trying to read 
> partitioned files from HDFS.
> I am running on Spark master branch #c544356
> The write actually works but the read fails.
> {code:title=Issue Reproduction}
> case class Data(id: Int, text: String)
> val ds = spark.createDataset( Seq(Data(0, "hello"), Data(1, "hello"), Data(0, 
> "world"), Data(1, "there")) )
> scala> 
> ds.write.mode(org.apache.spark.sql.SaveMode.Overwrite).format("parquet").partitionBy("id").save("/user/spark/test.parquet")
> SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".  
>   
> SLF4J: Defaulting to no-operation (NOP) logger implementation
> SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further 
> details.
> java.io.FileNotFoundException: Path is not a file: 
> /user/spark/test.parquet/id=0
> at 
> org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:75)
> at 
> org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:61)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocationsInt(FSNamesystem.java:1828)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1799)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1712)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getBlockLocations(NameNodeRpcServer.java:652)
> at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getBlockLocations(ClientNamenodeProtocolServerSideTranslatorPB.java:365)
> at 
> org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:616)
> at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:969)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2151)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2147)
> 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:1657)
> at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2145)
>   at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
>   at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
>   at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
>   at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
>   at 
> org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:106)
>   at 
> org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:73)
>   at 
> org.apache.hadoop.hdfs.DFSClient.callGetBlockLocations(DFSClient.java:1242)
>   at org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1227)
>   at org.apache.hadoop.hdfs.DFSClient.getBlockLocations(DFSClient.java:1285)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$1.doCall(DistributedFileSystem.java:221)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$1.doCall(DistributedFileSystem.java:217)
>   at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileBlockLocations(DistributedFileSystem.java:228)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileBlockLocations(DistributedFileSystem.java:209)
>   at 
> org.apache.spark.sql.execution.datasources.HDFSFileCatalog$$anonfun$9$$anonfun$apply$4.apply(fileSourceInterfaces.scala:372)
>   at 
> org.apache.spark.sql.execution.datasources.HDFSFileCatalog$$anonfun$9$$anonfun$apply$4.apply(fileSourceInterfaces.scala:360)
>   at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>   at 
> 

[jira] [Commented] (SPARK-14959) ​Problem Reading partitioned ORC or Parquet files

2016-05-11 Thread Sebastian YEPES FERNANDEZ (JIRA)

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

Sebastian YEPES FERNANDEZ commented on SPARK-14959:
---

[~sowen], The partitioned data exists and is readable by Spark, I can read it 
if I manually specify the partition:

{code}
scala> 
spark.read.format("parquet").load("hdfs://master:8020/user/spark/test.parquet/id=0").show
+-+
| text|
+-+
|hello|
|world|
+-+

scala> 
spark.read.format("parquet").load("hdfs://master:8020/user/spark/test.parquet/id=1").show
+-+
| text|
+-+
|hello|
|there|
+-+
{code}

> ​Problem Reading partitioned ORC or Parquet files
> -
>
> Key: SPARK-14959
> URL: https://issues.apache.org/jira/browse/SPARK-14959
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.0.0
> Environment: Hadoop 2.7.1.2.4.0.0-169 (HDP 2.4)
>Reporter: Sebastian YEPES FERNANDEZ
>
> Hello,
> I have noticed that in the pasts days there is an issue when trying to read 
> partitioned files from HDFS.
> I am running on Spark master branch #c544356
> The write actually works but the read fails.
> {code:title=Issue Reproduction}
> case class Data(id: Int, text: String)
> val ds = spark.createDataset( Seq(Data(0, "hello"), Data(1, "hello"), Data(0, 
> "world"), Data(1, "there")) )
> scala> 
> ds.write.mode(org.apache.spark.sql.SaveMode.Overwrite).format("parquet").partitionBy("id").save("/user/spark/test.parquet")
> SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".  
>   
> SLF4J: Defaulting to no-operation (NOP) logger implementation
> SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further 
> details.
> java.io.FileNotFoundException: Path is not a file: 
> /user/spark/test.parquet/id=0
> at 
> org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:75)
> at 
> org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:61)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocationsInt(FSNamesystem.java:1828)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1799)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1712)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getBlockLocations(NameNodeRpcServer.java:652)
> at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getBlockLocations(ClientNamenodeProtocolServerSideTranslatorPB.java:365)
> at 
> org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:616)
> at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:969)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2151)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2147)
> 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:1657)
> at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2145)
>   at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
>   at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
>   at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
>   at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
>   at 
> org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:106)
>   at 
> org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:73)
>   at 
> org.apache.hadoop.hdfs.DFSClient.callGetBlockLocations(DFSClient.java:1242)
>   at org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1227)
>   at org.apache.hadoop.hdfs.DFSClient.getBlockLocations(DFSClient.java:1285)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$1.doCall(DistributedFileSystem.java:221)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$1.doCall(DistributedFileSystem.java:217)
>   at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileBlockLocations(DistributedFileSystem.java:228)
>   at 
> 

[jira] [Commented] (SPARK-14959) ​Problem Reading partitioned ORC or Parquet files

2016-05-11 Thread Sean Owen (JIRA)

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

Sean Owen commented on SPARK-14959:
---

Does the data exist? is it readable? some basic debugging info like that would 
be useful

> ​Problem Reading partitioned ORC or Parquet files
> -
>
> Key: SPARK-14959
> URL: https://issues.apache.org/jira/browse/SPARK-14959
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.0.0
> Environment: Hadoop 2.7.1.2.4.0.0-169 (HDP 2.4)
>Reporter: Sebastian YEPES FERNANDEZ
>
> Hello,
> I have noticed that in the pasts days there is an issue when trying to read 
> partitioned files from HDFS.
> I am running on Spark master branch #c544356
> The write actually works but the read fails.
> {code:title=Issue Reproduction}
> case class Data(id: Int, text: String)
> val ds = spark.createDataset( Seq(Data(0, "hello"), Data(1, "hello"), Data(0, 
> "world"), Data(1, "there")) )
> scala> 
> ds.write.mode(org.apache.spark.sql.SaveMode.Overwrite).format("parquet").partitionBy("id").save("/user/spark/test.parquet")
> SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".  
>   
> SLF4J: Defaulting to no-operation (NOP) logger implementation
> SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further 
> details.
> java.io.FileNotFoundException: Path is not a file: 
> /user/spark/test.parquet/id=0
> at 
> org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:75)
> at 
> org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:61)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocationsInt(FSNamesystem.java:1828)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1799)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1712)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getBlockLocations(NameNodeRpcServer.java:652)
> at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getBlockLocations(ClientNamenodeProtocolServerSideTranslatorPB.java:365)
> at 
> org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:616)
> at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:969)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2151)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2147)
> 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:1657)
> at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2145)
>   at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
>   at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
>   at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
>   at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
>   at 
> org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:106)
>   at 
> org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:73)
>   at 
> org.apache.hadoop.hdfs.DFSClient.callGetBlockLocations(DFSClient.java:1242)
>   at org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1227)
>   at org.apache.hadoop.hdfs.DFSClient.getBlockLocations(DFSClient.java:1285)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$1.doCall(DistributedFileSystem.java:221)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$1.doCall(DistributedFileSystem.java:217)
>   at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileBlockLocations(DistributedFileSystem.java:228)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileBlockLocations(DistributedFileSystem.java:209)
>   at 
> org.apache.spark.sql.execution.datasources.HDFSFileCatalog$$anonfun$9$$anonfun$apply$4.apply(fileSourceInterfaces.scala:372)
>   at 
> org.apache.spark.sql.execution.datasources.HDFSFileCatalog$$anonfun$9$$anonfun$apply$4.apply(fileSourceInterfaces.scala:360)
>   at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>   at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>   at 
> 

[jira] [Commented] (SPARK-14959) ​Problem Reading partitioned ORC or Parquet files

2016-05-11 Thread Sebastian YEPES FERNANDEZ (JIRA)

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

Sebastian YEPES FERNANDEZ commented on SPARK-14959:
---

Hello [~sowen] using the full URL I still get the same error:

{code}
scala> 
spark.read.format("parquet").load("hdfs://master:8020/user/spark/test.parquet").show(1)
java.io.FileNotFoundException: Path is not a file: /user/spark/test.parquet/id=0
at 
org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:75)
at 
org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:61)
at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocationsInt(FSNamesystem.java:1828)
at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1799)
{code}


> ​Problem Reading partitioned ORC or Parquet files
> -
>
> Key: SPARK-14959
> URL: https://issues.apache.org/jira/browse/SPARK-14959
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.0.0
> Environment: Hadoop 2.7.1.2.4.0.0-169 (HDP 2.4)
>Reporter: Sebastian YEPES FERNANDEZ
>
> Hello,
> I have noticed that in the pasts days there is an issue when trying to read 
> partitioned files from HDFS.
> I am running on Spark master branch #c544356
> The write actually works but the read fails.
> {code:title=Issue Reproduction}
> case class Data(id: Int, text: String)
> val ds = spark.createDataset( Seq(Data(0, "hello"), Data(1, "hello"), Data(0, 
> "world"), Data(1, "there")) )
> scala> 
> ds.write.mode(org.apache.spark.sql.SaveMode.Overwrite).format("parquet").partitionBy("id").save("/user/spark/test.parquet")
> SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".  
>   
> SLF4J: Defaulting to no-operation (NOP) logger implementation
> SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further 
> details.
> java.io.FileNotFoundException: Path is not a file: 
> /user/spark/test.parquet/id=0
> at 
> org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:75)
> at 
> org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:61)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocationsInt(FSNamesystem.java:1828)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1799)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1712)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getBlockLocations(NameNodeRpcServer.java:652)
> at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getBlockLocations(ClientNamenodeProtocolServerSideTranslatorPB.java:365)
> at 
> org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:616)
> at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:969)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2151)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2147)
> 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:1657)
> at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2145)
>   at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
>   at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
>   at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
>   at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
>   at 
> org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:106)
>   at 
> org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:73)
>   at 
> org.apache.hadoop.hdfs.DFSClient.callGetBlockLocations(DFSClient.java:1242)
>   at org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1227)
>   at org.apache.hadoop.hdfs.DFSClient.getBlockLocations(DFSClient.java:1285)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$1.doCall(DistributedFileSystem.java:221)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$1.doCall(DistributedFileSystem.java:217)
>   at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>   at 
> 

[jira] [Commented] (SPARK-14959) ​Problem Reading partitioned ORC or Parquet files

2016-05-02 Thread Gang Wu (JIRA)

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

Gang Wu commented on SPARK-14959:
-

[~syepes] I faced the same exception when I try to query partitioned table on 
HDFS. Using the latest commit on master branch.

> ​Problem Reading partitioned ORC or Parquet files
> -
>
> Key: SPARK-14959
> URL: https://issues.apache.org/jira/browse/SPARK-14959
> Project: Spark
>  Issue Type: Bug
>  Components: Input/Output
>Affects Versions: 2.0.0
> Environment: Hadoop 2.7.1.2.4.0.0-169 (HDP 2.4)
>Reporter: Sebastian YEPES FERNANDEZ
>
> Hello,
> I have noticed that in the pasts days there is an issue when trying to read 
> partitioned files from HDFS.
> I am running on Spark master branch #c544356
> The write actually works but the read fails.
> {code:title=Issue Reproduction}
> case class Data(id: Int, text: String)
> val ds = spark.createDataset( Seq(Data(0, "hello"), Data(1, "hello"), Data(0, 
> "world"), Data(1, "there")) )
> scala> 
> ds.write.mode(org.apache.spark.sql.SaveMode.Overwrite).format("parquet").partitionBy("id").save("/user/spark/test.parquet")
> SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".  
>   
> SLF4J: Defaulting to no-operation (NOP) logger implementation
> SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further 
> details.
> java.io.FileNotFoundException: Path is not a file: 
> /user/spark/test.parquet/id=0
> at 
> org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:75)
> at 
> org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:61)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocationsInt(FSNamesystem.java:1828)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1799)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1712)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getBlockLocations(NameNodeRpcServer.java:652)
> at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getBlockLocations(ClientNamenodeProtocolServerSideTranslatorPB.java:365)
> at 
> org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:616)
> at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:969)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2151)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2147)
> 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:1657)
> at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2145)
>   at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
>   at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
>   at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
>   at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
>   at 
> org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:106)
>   at 
> org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:73)
>   at 
> org.apache.hadoop.hdfs.DFSClient.callGetBlockLocations(DFSClient.java:1242)
>   at org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1227)
>   at org.apache.hadoop.hdfs.DFSClient.getBlockLocations(DFSClient.java:1285)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$1.doCall(DistributedFileSystem.java:221)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$1.doCall(DistributedFileSystem.java:217)
>   at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileBlockLocations(DistributedFileSystem.java:228)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileBlockLocations(DistributedFileSystem.java:209)
>   at 
> org.apache.spark.sql.execution.datasources.HDFSFileCatalog$$anonfun$9$$anonfun$apply$4.apply(fileSourceInterfaces.scala:372)
>   at 
> org.apache.spark.sql.execution.datasources.HDFSFileCatalog$$anonfun$9$$anonfun$apply$4.apply(fileSourceInterfaces.scala:360)
>   at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>   at 
> 

[jira] [Commented] (SPARK-14959) ​Problem Reading partitioned ORC or Parquet files

2016-04-28 Thread Sebastian YEPES FERNANDEZ (JIRA)

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

Sebastian YEPES FERNANDEZ commented on SPARK-14959:
---

[~bomeng] I have just retested it with the last master commit 
be317d4a90b3ca906fefeb438f89a09b1c7da5a8 and I am still getting the same error.

have you tested this with HDFS?

{code:title=spakr-shell}
scala> 
ds.write.mode(org.apache.spark.sql.SaveMode.Overwrite).format("parquet").partitionBy("id").save("/user/spark/test.parquet")
SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
SLF4J: Defaulting to no-operation (NOP) logger implementation
SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further 
details.
java.io.FileNotFoundException: Path is not a file: /user/spark/test.parquet/id=0
at 
org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:75)
at 
org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:61)
at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocationsInt(FSNamesystem.java:1828)
at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1799)
...
...
  at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
  at java.lang.reflect.Method.invoke(Method.java:498)
  at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:187)
  at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
  at com.sun.proxy.$Proxy14.getBlockLocations(Unknown Source)
  at org.apache.hadoop.hdfs.DFSClient.callGetBlockLocations(DFSClient.java:1240)
  ... 78 more
{code}

> ​Problem Reading partitioned ORC or Parquet files
> -
>
> Key: SPARK-14959
> URL: https://issues.apache.org/jira/browse/SPARK-14959
> Project: Spark
>  Issue Type: Bug
>  Components: Input/Output
>Affects Versions: 2.0.0
> Environment: Hadoop 2.7.1.2.4.0.0-169 (HDP 2.4)
>Reporter: Sebastian YEPES FERNANDEZ
>Priority: Critical
>
> Hello,
> I have noticed that in the pasts days there is an issue when trying to read 
> partitioned files from HDFS.
> I am running on Spark master branch #c544356
> The write actually works but the read fails.
> {code:title=Issue Reproduction}
> case class Data(id: Int, text: String)
> val ds = spark.createDataset( Seq(Data(0, "hello"), Data(1, "hello"), Data(0, 
> "world"), Data(1, "there")) )
> scala> 
> ds.write.mode(org.apache.spark.sql.SaveMode.Overwrite).format("parquet").partitionBy("id").save("/user/spark/test.parquet")
> SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".  
>   
> SLF4J: Defaulting to no-operation (NOP) logger implementation
> SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further 
> details.
> java.io.FileNotFoundException: Path is not a file: 
> /user/spark/test.parquet/id=0
> at 
> org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:75)
> at 
> org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:61)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocationsInt(FSNamesystem.java:1828)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1799)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1712)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getBlockLocations(NameNodeRpcServer.java:652)
> at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getBlockLocations(ClientNamenodeProtocolServerSideTranslatorPB.java:365)
> at 
> org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:616)
> at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:969)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2151)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2147)
> 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:1657)
> at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2145)
>   at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
>   at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
>   at 
> 

[jira] [Commented] (SPARK-14959) ​Problem Reading partitioned ORC or Parquet files

2016-04-27 Thread Bo Meng (JIRA)

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

Bo Meng commented on SPARK-14959:
-

I have tried on master branch, it works fine with the latest code. 

> ​Problem Reading partitioned ORC or Parquet files
> -
>
> Key: SPARK-14959
> URL: https://issues.apache.org/jira/browse/SPARK-14959
> Project: Spark
>  Issue Type: Bug
>  Components: Input/Output
>Affects Versions: 2.0.0
> Environment: Hadoop 2.7.1.2.4.0.0-169 (HDP 2.4)
>Reporter: Sebastian YEPES FERNANDEZ
>Priority: Critical
>
> Hello,
> I have noticed that in the pasts days there is an issue when trying to read 
> partitioned files from HDFS.
> I am running on Spark master branch #c544356
> The write actually works but the read fails.
> {code:title=Issue Reproduction}
> case class Data(id: Int, text: String)
> val ds = spark.createDataset( Seq(Data(0, "hello"), Data(1, "hello"), Data(0, 
> "world"), Data(1, "there")) )
> scala> 
> ds.write.mode(org.apache.spark.sql.SaveMode.Overwrite).format("parquet").partitionBy("id").save("/user/spark/test.parquet")
> SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".  
>   
> SLF4J: Defaulting to no-operation (NOP) logger implementation
> SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further 
> details.
> java.io.FileNotFoundException: Path is not a file: 
> /user/spark/test.parquet/id=0
> at 
> org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:75)
> at 
> org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:61)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocationsInt(FSNamesystem.java:1828)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1799)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1712)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getBlockLocations(NameNodeRpcServer.java:652)
> at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getBlockLocations(ClientNamenodeProtocolServerSideTranslatorPB.java:365)
> at 
> org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:616)
> at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:969)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2151)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2147)
> 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:1657)
> at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2145)
>   at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
>   at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
>   at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
>   at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
>   at 
> org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:106)
>   at 
> org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:73)
>   at 
> org.apache.hadoop.hdfs.DFSClient.callGetBlockLocations(DFSClient.java:1242)
>   at org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1227)
>   at org.apache.hadoop.hdfs.DFSClient.getBlockLocations(DFSClient.java:1285)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$1.doCall(DistributedFileSystem.java:221)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$1.doCall(DistributedFileSystem.java:217)
>   at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileBlockLocations(DistributedFileSystem.java:228)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileBlockLocations(DistributedFileSystem.java:209)
>   at 
> org.apache.spark.sql.execution.datasources.HDFSFileCatalog$$anonfun$9$$anonfun$apply$4.apply(fileSourceInterfaces.scala:372)
>   at 
> org.apache.spark.sql.execution.datasources.HDFSFileCatalog$$anonfun$9$$anonfun$apply$4.apply(fileSourceInterfaces.scala:360)
>   at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>   at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>   at 
>