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

Permission problem happens while prepareLocalResources

    XMLWordPrintableJSON

Details

    • Improvement
    • Status: In Progress
    • Major
    • Resolution: Unresolved
    • 2.2.0, 2.3.2, 2.4.0, 3.0.0
    • None
    • YARN
    • None

    Description

      if spark.yarn.archive and spark.yarn.stagingDir points to a different file system, the directory of spark.yarn.archive needs to be upload to a destPath(started with spark.yarn.stagingDir). But the default permission set to the destPath is 644(APP_FILE_PERMISSION). This permission is OK with file case, but not with directory.

      val sparkArchive = sparkConf.get(SPARK_ARCHIVE)
      if (sparkArchive.isDefined) {
        val archive = sparkArchive.get
        require(!Utils.isLocalUri(archive), s"${SPARK_ARCHIVE.key} cannot be a local URI.")
        distribute(Utils.resolveURI(archive).toString,
          resType = LocalResourceType.ARCHIVE,
          destName = Some(LOCALIZED_LIB_DIR))
      }
      
       if (force || !compareFs(srcFs, destFs) || "file".equals(srcFs.getScheme)) { destPath = new Path(destDir, destName.getOrElse(srcPath.getName())) logInfo(s"Uploading resource $srcPath -> $destPath") FileUtil.copy(srcFs, srcPath, destFs, destPath, false, hadoopConf) destFs.setReplication(destPath, replication) if (destFs.getFileStatus(destPath).isDirectory) { destFs.setPermission(destPath, new FsPermission(APP_DIRECTORY_PERMISSION)) } else { destFs.setPermission(destPath, new FsPermission(APP_FILE_PERMISSION)) } } else { logInfo(s"Source and destination file systems are the same. Not copying $srcPath") }
      
      Coping the spark.yarn.archive directory

      2020-07-20 14:53:36,488 [main] INFO org.apache.spark.deploy.yarn.Client - Setting up container launch context for our AM 2020-07-20 14:53:36,490 [main] INFO org.apache.spark.deploy.yarn.Client - Setting up the launch environment for our AM container 2020-07-20 14:53:36,494 [main] INFO org.apache.spark.deploy.yarn.Client - Preparing resources for our AM container 2020-07-20 14:53:36,582 [main] INFO org.apache.spark.deploy.yarn.Client - Uploading resource hdfs://n-fed/user/oozie/share/lib/lib_2020060819522/sparksql -> hdfs://test-hadoop/data/spark/stagingDir/.sparkStaging/application_1591344376643_55140/sparksql 2020-07-20 14:53:48,171 [main] INFO org.apache.spark.deploy.yarn.Client - Uploading resource hdfs://n-fed/user/oozie/share/lib/lib_2020060819522/sparksql/javax.inject-2.4.0-b34.jar -> hdfs://test-hadoop/data/spark/stagingDir/.sparkStaging/application_1591344376643_55140/javax.inject-2.4.0-b34.ja

      Error happened

      Diagnostics: Permission denied: user=jsq, access=READ_EXECUTE, inode="/data/spark/stagingDir/.sparkStaging/application_1591344376643_55140/sparksql":jsq:hdfs:drw-r-r- at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.check(FSPermissionChecker.java:319) at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.checkPermission(FSPermissionChecker.java:219) at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.checkPermission(FSPermissionChecker.java:190) at org.apache.hadoop.hdfs.server.namenode.FSDirectory.checkPermission(FSDirectory.java:1780) at org.apache.hadoop.hdfs.server.namenode.FSDirectory.checkPermission(FSDirectory.java:1764) at org.apache.hadoop.hdfs.server.namenode.FSDirectory.checkPathAccess(FSDirectory.java:1738) at org.apache.hadoop.hdfs.server.namenode.FSDirStatAndListingOp.getListingInt(FSDirStatAndListingOp.java:76) at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getListing(FSNamesystem.java:4565) at org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getListing(NameNodeRpcServer.java:1104) at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getListing(ClientNamenodeProtocolServerSideTranslatorPB.java:642) 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:982) at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2211) at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2207) 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:1724) at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2205)

      Attachments

        Activity

          People

            Unassigned Unassigned
            sekingme jiangshequan
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated: