You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by yl...@apache.org on 2016/11/22 08:05:42 UTC

spark git commit: [SPARK-18444][SPARKR] SparkR running in yarn-cluster mode should not download Spark package.

Repository: spark
Updated Branches:
  refs/heads/master ebeb0830a -> acb971577


[SPARK-18444][SPARKR] SparkR running in yarn-cluster mode should not download Spark package.

## What changes were proposed in this pull request?
When running SparkR job in yarn-cluster mode, it will download Spark package from apache website which is not necessary.
```
./bin/spark-submit --master yarn-cluster ./examples/src/main/r/dataframe.R
```
The following is output:
```
Attaching package: \u2018SparkR\u2019

The following objects are masked from \u2018package:stats\u2019:

    cov, filter, lag, na.omit, predict, sd, var, window

The following objects are masked from \u2018package:base\u2019:

    as.data.frame, colnames, colnames<-, drop, endsWith, intersect,
    rank, rbind, sample, startsWith, subset, summary, transform, union

Spark not found in SPARK_HOME:
Spark not found in the cache directory. Installation will start.
MirrorUrl not provided.
Looking for preferred site from apache website...
......
```
There's no ```SPARK_HOME``` in yarn-cluster mode since the R process is in a remote host of the yarn cluster rather than in the client host. The JVM comes up first and the R process then connects to it. So in such cases we should never have to download Spark as Spark is already running.

## How was this patch tested?
Offline test.

Author: Yanbo Liang <yb...@gmail.com>

Closes #15888 from yanboliang/spark-18444.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/acb97157
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/acb97157
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/acb97157

Branch: refs/heads/master
Commit: acb97157796231fef74aba985825b05b607b9279
Parents: ebeb083
Author: Yanbo Liang <yb...@gmail.com>
Authored: Tue Nov 22 00:05:30 2016 -0800
Committer: Yanbo Liang <yb...@gmail.com>
Committed: Tue Nov 22 00:05:30 2016 -0800

----------------------------------------------------------------------
 R/pkg/R/sparkR.R                        | 20 ++++++++----
 R/pkg/R/utils.R                         |  4 +++
 R/pkg/inst/tests/testthat/test_sparkR.R | 46 ++++++++++++++++++++++++++++
 3 files changed, 64 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/acb97157/R/pkg/R/sparkR.R
----------------------------------------------------------------------
diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R
index 6b4a2f2..a7152b4 100644
--- a/R/pkg/R/sparkR.R
+++ b/R/pkg/R/sparkR.R
@@ -373,8 +373,13 @@ sparkR.session <- function(
     overrideEnvs(sparkConfigMap, paramMap)
   }
 
+  deployMode <- ""
+  if (exists("spark.submit.deployMode", envir = sparkConfigMap)) {
+    deployMode <- sparkConfigMap[["spark.submit.deployMode"]]
+  }
+
   if (!exists(".sparkRjsc", envir = .sparkREnv)) {
-    retHome <- sparkCheckInstall(sparkHome, master)
+    retHome <- sparkCheckInstall(sparkHome, master, deployMode)
     if (!is.null(retHome)) sparkHome <- retHome
     sparkExecutorEnvMap <- new.env()
     sparkR.sparkContext(master, appName, sparkHome, sparkConfigMap, sparkExecutorEnvMap,
@@ -550,24 +555,27 @@ processSparkPackages <- function(packages) {
 #
 # @param sparkHome directory to find Spark package.
 # @param master the Spark master URL, used to check local or remote mode.
+# @param deployMode whether to deploy your driver on the worker nodes (cluster)
+#        or locally as an external client (client).
 # @return NULL if no need to update sparkHome, and new sparkHome otherwise.
-sparkCheckInstall <- function(sparkHome, master) {
+sparkCheckInstall <- function(sparkHome, master, deployMode) {
   if (!isSparkRShell()) {
     if (!is.na(file.info(sparkHome)$isdir)) {
       msg <- paste0("Spark package found in SPARK_HOME: ", sparkHome)
       message(msg)
       NULL
     } else {
-      if (!nzchar(master) || isMasterLocal(master)) {
-        msg <- paste0("Spark not found in SPARK_HOME: ",
-                      sparkHome)
+      if (isMasterLocal(master)) {
+        msg <- paste0("Spark not found in SPARK_HOME: ", sparkHome)
         message(msg)
         packageLocalDir <- install.spark()
         packageLocalDir
-      } else {
+      } else if (isClientMode(master) || deployMode == "client") {
         msg <- paste0("Spark not found in SPARK_HOME: ",
                       sparkHome, "\n", installInstruction("remote"))
         stop(msg)
+      } else {
+        NULL
       }
     }
   } else {

http://git-wip-us.apache.org/repos/asf/spark/blob/acb97157/R/pkg/R/utils.R
----------------------------------------------------------------------
diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R
index 2000454..098c0e3 100644
--- a/R/pkg/R/utils.R
+++ b/R/pkg/R/utils.R
@@ -777,6 +777,10 @@ isMasterLocal <- function(master) {
   grepl("^local(\\[([0-9]+|\\*)\\])?$", master, perl = TRUE)
 }
 
+isClientMode <- function(master) {
+  grepl("([a-z]+)-client$", master, perl = TRUE)
+}
+
 isSparkRShell <- function() {
   grepl(".*shell\\.R$", Sys.getenv("R_PROFILE_USER"), perl = TRUE)
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/acb97157/R/pkg/inst/tests/testthat/test_sparkR.R
----------------------------------------------------------------------
diff --git a/R/pkg/inst/tests/testthat/test_sparkR.R b/R/pkg/inst/tests/testthat/test_sparkR.R
new file mode 100644
index 0000000..f73fc6b
--- /dev/null
+++ b/R/pkg/inst/tests/testthat/test_sparkR.R
@@ -0,0 +1,46 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+context("functions in sparkR.R")
+
+test_that("sparkCheckInstall", {
+  # "local, yarn-client, mesos-client" mode, SPARK_HOME was set correctly,
+  # and the SparkR job was submitted by "spark-submit"
+  sparkHome <- paste0(tempdir(), "/", "sparkHome")
+  dir.create(sparkHome)
+  master <- ""
+  deployMode <- ""
+  expect_true(is.null(sparkCheckInstall(sparkHome, master, deployMode)))
+  unlink(sparkHome, recursive = TRUE)
+
+  # "yarn-cluster, mesos-cluster" mode, SPARK_HOME was not set,
+  # and the SparkR job was submitted by "spark-submit"
+  sparkHome <- ""
+  master <- ""
+  deployMode <- ""
+  expect_true(is.null(sparkCheckInstall(sparkHome, master, deployMode)))
+
+  # "yarn-client, mesos-client" mode, SPARK_HOME was not set
+  sparkHome <- ""
+  master <- "yarn-client"
+  deployMode <- ""
+  expect_error(sparkCheckInstall(sparkHome, master, deployMode))
+  sparkHome <- ""
+  master <- ""
+  deployMode <- "client"
+  expect_error(sparkCheckInstall(sparkHome, master, deployMode))
+})


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