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

Thomas Graves commented on SPARK-14331:
---------------------------------------

I was trying to reproduce this to get you the rest but now I get a different 
exception, this is using sqlContext directly: 
    val sqlContext = new org.apache.spark.sql.SQLContext(sc)

It looks like its perhaps using the wrong filesystem (using hadoop when it 
should use local) .

16/05/23 14:15:08 ERROR ApplicationMaster: User class threw exception: 
org.apache.spark.SparkException: Unable to create database default as failed to 
create its directory 
hdfs://nn1.com:8020/hadoop/tmp/yarn-local/usercache/tgraves/appcache/application_1463805142339_520258/container_e11_1463805142339_520258_01_000001/spark-warehouse
org.apache.spark.SparkException: Unable to create database default as failed to 
create its directory 
hdfs://nn1.com:8020/hadoop/tmp/yarn-local/usercache/tgraves/appcache/application_1463805142339_520258/container_e11_1463805142339_520258_01_000001/spark-warehouse
        at 
org.apache.spark.sql.catalyst.catalog.InMemoryCatalog.liftedTree1$1(InMemoryCatalog.scala:126)
        at 
org.apache.spark.sql.catalyst.catalog.InMemoryCatalog.createDatabase(InMemoryCatalog.scala:122)
        at 
org.apache.spark.sql.catalyst.catalog.SessionCatalog.createDatabase(SessionCatalog.scala:142)
        at 
org.apache.spark.sql.catalyst.catalog.SessionCatalog.<init>(SessionCatalog.scala:84)
        at 
org.apache.spark.sql.internal.SessionState.catalog$lzycompute(SessionState.scala:94)
        at 
org.apache.spark.sql.internal.SessionState.catalog(SessionState.scala:94)
        at 
org.apache.spark.sql.internal.SessionState$$anon$1.<init>(SessionState.scala:110)
        at 
org.apache.spark.sql.internal.SessionState.analyzer$lzycompute(SessionState.scala:110)
        at 
org.apache.spark.sql.internal.SessionState.analyzer(SessionState.scala:109)
        at 
org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:48)
        at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:62)
        at 
org.apache.spark.sql.SparkSession.baseRelationToDataFrame(SparkSession.scala:383)
        at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:154)
        at 
org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:419)
        at yahoo.spark.SparkFlickrLargeJoin$.main(SparkFlickrLargeJoin.scala:26)
        at yahoo.spark.SparkFlickrLargeJoin.main(SparkFlickrLargeJoin.scala)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:497)
        at 
org.apache.spark.deploy.yarn.ApplicationMaster$$anon$2.run(ApplicationMaster.scala:617)
Caused by: org.apache.hadoop.security.AccessControlException: Permission 
denied: user=tgraves, access=WRITE, inode="/":hdfs:hdfs:drwxr-xr-x
        at 
org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.check(FSPermissionChecker.java:298)
        at 
org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.checkPermission(FSPermissionChecker.java:204)
        at 
org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.checkPermission(FSPermissionChecker.java:182)
        at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem$PathResolver.verifyPermissions(FSNamesystem.java:8622)
        at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.mkdirs(FSNamesystem.java:3961)
        at 
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.mkdirs(NameNodeRpcServer.java:989)
        at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.mkdirs(ClientNamenodeProtocolServerSideTranslatorPB.java:622)
        at 
org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
        at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:608)
        at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:982)
        at org.apache.hadoop.ipc.Server.call(Server.java:2267)
        at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:648)
        at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:615)
        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:1720)
        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2217)

        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:422)
        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.primitiveMkdir(DFSClient.java:3018)
        at org.apache.hadoop.hdfs.DFSClient.mkdirs(DFSClient.java:2986)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem$21.doCall(DistributedFileSystem.java:1047)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem$21.doCall(DistributedFileSystem.java:1043)
        at 
org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem.mkdirsInternal(DistributedFileSystem.java:1061)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem.mkdirs(DistributedFileSystem.java:1036)
        at org.apache.hadoop.fs.FileSystem.mkdirs(FileSystem.java:1880)
        at 
org.apache.spark.sql.catalyst.catalog.InMemoryCatalog.liftedTree1$1(InMemoryCatalog.scala:123)
        ... 20 more
Caused by: 
org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.security.AccessControlException):
 Permission denied: user=tgraves, access=WRITE, inode="/":hdfs:hdfs:drwxr-xr-x
        at 
org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.check(FSPermissionChecker.java:298)
        at 
org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.checkPermission(FSPermissionChecker.java:204)
        at 
org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.checkPermission(FSPermissionChecker.java:182)
        at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem$PathResolver.verifyPermissions(FSNamesystem.java:8622)
        at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.mkdirs(FSNamesystem.java:3961)
        at 
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.mkdirs(NameNodeRpcServer.java:989)
        at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.mkdirs(ClientNamenodeProtocolServerSideTranslatorPB.java:622)
        at 
org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
        at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:608)
        at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:982)
        at org.apache.hadoop.ipc.Server.call(Server.java:2267)
        at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:648)
        at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:615)
        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:1720)
        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2217)

        at org.apache.hadoop.ipc.Client.call(Client.java:1471)
        at org.apache.hadoop.ipc.Client.call(Client.java:1408)
        at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:230)
        at com.sun.proxy.$Proxy10.mkdirs(Unknown Source)
        at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.mkdirs(ClientNamenodeProtocolTranslatorPB.java:558)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:497)
        at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:191)
        at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
        at com.sun.proxy.$Proxy11.mkdirs(Unknown Source)
        at org.apache.hadoop.hdfs.DFSClient.primitiveMkdir(DFSClient.java:3016)
        ... 28 more

> Exceptions saving to parquetFile after join from dataframes in master
> ---------------------------------------------------------------------
>
>                 Key: SPARK-14331
>                 URL: https://issues.apache.org/jira/browse/SPARK-14331
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 2.0.0
>            Reporter: Thomas Graves
>            Priority: Critical
>
> I'm trying to use master and write to a parquet file when using a dataframe 
> but am seeing the exception below.  Not sure exact state of dataframes right 
> now so if this is known issue let me know.
> I read 2 sources of parquet files, joined them, then saved them back.
>  val df_pixels = sqlContext.read.parquet("data1")
>     val df_pixels_renamed = df_pixels.withColumnRenamed("photo_id", 
> "pixels_photo_id")
>     val df_meta = sqlContext.read.parquet("data2")
>     val df = df_meta.as("meta").join(df_pixels_renamed, $"meta.photo_id" === 
> $"pixels_photo_id", "inner").drop("pixels_photo_id")
>     df.write.parquet(args(0))
> 16/04/01 17:21:34 ERROR InsertIntoHadoopFsRelation: Aborting job.
> org.apache.spark.sql.catalyst.errors.package$TreeNodeException: execute, tree:
> Exchange hashpartitioning(pixels_photo_id#3, 20000), None
> +- WholeStageCodegen
>    :  +- Filter isnotnull(pixels_photo_id#3)
>    :     +- INPUT
>    +- Coalesce 0
>       +- WholeStageCodegen
>          :  +- Project [img_data#0,photo_id#1 AS pixels_photo_id#3]
>          :     +- Scan HadoopFiles[img_data#0,photo_id#1] Format: 
> ParquetFormat, PushedFilters: [], ReadSchema: 
> struct<img_data:binary,photo_id:string>
>         at 
> org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:47)
>         at 
> org.apache.spark.sql.execution.exchange.ShuffleExchange.doExecute(ShuffleExchange.scala:109)
>         at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:118)
>         at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:118)
>         at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:137)
>         at 
> org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
>         at 
> org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:134)
>         at 
> org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:117)
>         at 
> org.apache.spark.sql.execution.InputAdapter.upstreams(WholeStageCodegen.scala:236)
>         at org.apache.spark.sql.execution.Sort.upstreams(Sort.scala:104)
>         at 
> org.apache.spark.sql.execution.WholeStageCodegen.doExecute(WholeStageCodegen.scala:351)
>         at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:118)
>         at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:118)
>         at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:137)
>         at 
> org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
>         at 
> org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:134)
>         at 
> org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:117)
>         at 
> org.apache.spark.sql.execution.InputAdapter.doExecute(WholeStageCodegen.scala:228)
>         at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:118)
>         at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:118)
>         at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:137)
>         at 
> org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
>         at 
> org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:134)
>         at 
> org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:117)



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

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

Reply via email to