spark-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Shao, Saisai" <saisai.s...@intel.com>
Subject RE: Executing spark jobs with predefined Hadoop user
Date Thu, 10 Apr 2014 14:33:21 GMT
Hi Asaf,

The user who run SparkContext is decided by the below code in SparkContext, normally this
user.name is the user who started JVM, you can start your application with -Duser.name=xxx
to specify a username you want, this specified username will be the user to communicate with
HDFS.

 val sparkUser = Option {
    Option(System.getProperty("user.name")).getOrElse(System.getenv("SPARK_USER"))
  }.getOrElse {
    SparkContext.SPARK_UNKNOWN_USER
  }

Thanks
Jerry

From: Asaf Lahav [mailto:asaf.lahav@gmail.com]
Sent: Thursday, April 10, 2014 8:15 PM
To: user@spark.apache.org
Subject: Executing spark jobs with predefined Hadoop user

Hi,
We are using Spark with data files on HDFS. The files are stored as files for predefined hadoop
user ("hdfs").
The folder is permitted with
*         read write, executable and read permission for the hdfs user
*         executable and read permission for users in the group
*         just read permission for all other users

now the Spark write operation fails, due to a user mismatch of the spark context and the Hadoop
user permission.
Is there a way to start the Spark Context with another user than the one configured on the
local machine?



Please the technical details below:





The permission on the hdfs folder "/tmp/Iris" is as follows:
drwxr-xr-x   - hdfs      hadoop          0 2014-04-10 14:12 /tmp/Iris


The Spark context is initiated on my local machine and according to the configured hdfs permission
"rwxr-xr-x" there is no problem in loading the Hadoop hdfs file into a rdd:
final JavaRDD<String> rdd = sparkContext.textFile(filePath);

But saving the resulted rdd back to Hadoop resulst in an Hadoop security exception:
rdd.saveAsTextFile("/tmp/Iris/output");

Then the I receive the following Hadoop security exception:
org.apache.hadoop.security.AccessControlException: org.apache.hadoop.security.AccessControlException:
Permission denied: user=halbani, access=WRITE, inode="/tmp/Iris":hdfs:hadoop:drwxr-xr-x
          at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
          at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:57)
          at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
          at java.lang.reflect.Constructor.newInstance(Constructor.java:525)
          at org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:95)
          at org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:57)
          at org.apache.hadoop.hdfs.DFSClient.mkdirs(DFSClient.java:1428)
          at org.apache.hadoop.hdfs.DistributedFileSystem.mkdirs(DistributedFileSystem.java:332)
          at org.apache.hadoop.fs.FileSystem.mkdirs(FileSystem.java:1126)
          at org.apache.hadoop.mapred.FileOutputCommitter.setupJob(FileOutputCommitter.java:52)
          at org.apache.hadoop.mapred.SparkHadoopWriter.preSetup(SparkHadoopWriter.scala:65)
          at org.apache.spark.rdd.PairRDDFunctions.saveAsHadoopDataset(PairRDDFunctions.scala:713)
          at org.apache.spark.rdd.PairRDDFunctions.saveAsHadoopFile(PairRDDFunctions.scala:686)
          at org.apache.spark.rdd.PairRDDFunctions.saveAsHadoopFile(PairRDDFunctions.scala:572)
          at org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:894)
          at org.apache.spark.api.java.JavaRDDLike$class.saveAsTextFile(JavaRDDLike.scala:355)
          at org.apache.spark.api.java.JavaRDD.saveAsTextFile(JavaRDD.scala:27)
          at org.apache.spark.reader.FileSpliter.split(FileSpliter.java:73)
          at org.apache.spark.reader.FileReaderMain.main(FileReaderMain.java:17)
          at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
          at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
          at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
          at java.lang.reflect.Method.invoke(Method.java:601)
          at com.intellij.rt.execution.application.AppMain.main(AppMain.java:120)
Caused by: org.apache.hadoop.ipc.RemoteException: org.apache.hadoop.security.AccessControlException:
Permission denied: user=halbani, access=WRITE, inode="/tmp/Iris":hdfs:hadoop:drwxr-xr-x
          at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.check(FSPermissionChecker.java:225)
          at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.check(FSPermissionChecker.java:205)
          at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.checkPermission(FSPermissionChecker.java:151)
          at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkPermission(FSNamesystem.java:5951)
          at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkAncestorAccess(FSNamesystem.java:5924)
          at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.mkdirsInternal(FSNamesystem.java:2628)
          at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.mkdirs(FSNamesystem.java:2593)
          at org.apache.hadoop.hdfs.server.namenode.NameNode.mkdirs(NameNode.java:927)
          at sun.reflect.GeneratedMethodAccessor132.invoke(Unknown Source)
          at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
          at java.lang.reflect.Method.invoke(Method.java:606)
          at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:587)
          at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1444)
          at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1440)
          at java.security.AccessController.doPrivileged(Native Method)
          at javax.security.auth.Subject.doAs(Subject.java:415)
          at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1232)
          at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1438)

          at org.apache.hadoop.ipc.Client.call(Client.java:1107)
          at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:229)
          at $Proxy7.mkdirs(Unknown Source)
          at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
          at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
          at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
          at java.lang.reflect.Method.invoke(Method.java:601)
          at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:85)
          at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:62)
          at $Proxy7.mkdirs(Unknown Source)
          at org.apache.hadoop.hdfs.DFSClient.mkdirs(DFSClient.java:1426)
          ... 17 more


Apparently is the Spark context is initiated with the user on the local machine.
Is there a way to start the Spark Context with another user then the one configured on the
local machine?

Mime
View raw message