You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by WangTaoTheTonic <gi...@git.apache.org> on 2015/02/10 14:48:47 UTC

[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

GitHub user WangTaoTheTonic opened a pull request:

    https://github.com/apache/spark/pull/4505

    [SPARK-5719]allow daemons to bind to specified host

    https://issues.apache.org/jira/browse/SPARK-5719
    
    Add four config items to specify the host that daemons bind to:
    
    spark.master.host
    spark.worker.host
    spark.ui.host
    spark.history.ui.host
    
    The added config items also work for daemons.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/WangTaoTheTonic/spark SPARK-5719

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/4505.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #4505
    
----
commit 14424471d8332aac6211d0e8e9b9cd11b7d473f4
Author: WangTaoTheTonic <wa...@huawei.com>
Date:   2015-02-10T13:23:07Z

    allow UIs to bind to specified host

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/4505#issuecomment-75911609
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/27940/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/4505#discussion_r28116221
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala ---
    @@ -23,46 +23,56 @@ import org.apache.spark.util.{IntParam, MemoryParam, Utils}
     import org.apache.spark.SparkConf
     
     /**
    - * Command-line parser for the worker.
    + * Arguments parser for the worker.
      */
     private[spark] class WorkerArguments(args: Array[String], conf: SparkConf) {
    -  var host = Utils.localHostName()
    -  var port = 0
    -  var webUiPort = 8081
    -  var cores = inferDefaultCores()
    -  var memory = inferDefaultMemory()
    +  var host: String = null
    +  var port: Int = -1
    +  var webUiPort: Int = -1
    +  var cores: Int = -1
    +  var memory: Int = -1
    --- End diff --
    
    ah I see


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/4505#issuecomment-73701822
  
      [Test build #27206 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27206/consoleFull) for   PR 4505 at commit [`1442447`](https://github.com/apache/spark/commit/14424471d8332aac6211d0e8e9b9cd11b7d473f4).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on the pull request:

    https://github.com/apache/spark/pull/4505#issuecomment-91536236
  
    @WangTaoTheTonic Isn't this what things like `SPARK_MASTER_IP` are for?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/4505#issuecomment-91391523
  
    I'm still not sure if this is a use case we should support. Maybe others @JoshRosen @srowen have some thoughts.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/4505#issuecomment-73710872
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/27206/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

Posted by WangTaoTheTonic <gi...@git.apache.org>.
Github user WangTaoTheTonic commented on the pull request:

    https://github.com/apache/spark/pull/4505#issuecomment-75891305
  
    Assume that machines in our cluster have two network interfaces, one is used for management mainly while the other is used for business. Spark is the only one of many components and we just wanna access its Web UI via one of the network interfaces. That makes configurable ip address necessary.
    
    Most configuration could be assigned with a wrong value, it will be ok as long as we give an error prompt when user configure it improperly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/4505#issuecomment-75106654
  
    @WangTaoTheTonic what is the motivation for binding to a different host other than `0.0.0.0`? IIUC binding to `0.0.0.0` means you bind to all local interfaces. If we make this configurable then we're essentially allowing the user to bind to an address that is not part of this machine, which doesn't make sense.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/4505#issuecomment-73738163
  
      [Test build #27212 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27212/consoleFull) for   PR 4505 at commit [`3a768d4`](https://github.com/apache/spark/commit/3a768d4419745cd7ceb56e3b4648d5de7c35daf4).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

Posted by WangTaoTheTonic <gi...@git.apache.org>.
Github user WangTaoTheTonic commented on a diff in the pull request:

    https://github.com/apache/spark/pull/4505#discussion_r25316462
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala ---
    @@ -23,46 +23,56 @@ import org.apache.spark.util.{IntParam, MemoryParam, Utils}
     import org.apache.spark.SparkConf
     
     /**
    - * Command-line parser for the worker.
    + * Arguments parser for the worker.
      */
     private[spark] class WorkerArguments(args: Array[String], conf: SparkConf) {
    -  var host = Utils.localHostName()
    -  var port = 0
    -  var webUiPort = 8081
    -  var cores = inferDefaultCores()
    -  var memory = inferDefaultMemory()
    +  var host: String = null
    +  var port: Int = -1
    +  var webUiPort: Int = -1
    +  var cores: Int = -1
    +  var memory: Int = -1
    --- End diff --
    
    How could we know whether `port` is already assigned with the value after `--port`(in the function `parse`)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

Posted by WangTaoTheTonic <gi...@git.apache.org>.
Github user WangTaoTheTonic closed the pull request at:

    https://github.com/apache/spark/pull/4505


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/4505#discussion_r25010786
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala ---
    @@ -23,46 +23,56 @@ import org.apache.spark.util.{IntParam, MemoryParam, Utils}
     import org.apache.spark.SparkConf
     
     /**
    - * Command-line parser for the worker.
    + * Arguments parser for the worker.
      */
     private[spark] class WorkerArguments(args: Array[String], conf: SparkConf) {
    -  var host = Utils.localHostName()
    -  var port = 0
    -  var webUiPort = 8081
    -  var cores = inferDefaultCores()
    -  var memory = inferDefaultMemory()
    +  var host: String = null
    +  var port: Int = -1
    +  var webUiPort: Int = -1
    +  var cores: Int = -1
    +  var memory: Int = -1
    --- End diff --
    
    you don't need to change the defaults here if you do the following down there:
    ```
    port = sys.env.get(...)
      .orElse(conf.getOption(...))
      .getOrElse(port)
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

Posted by WangTaoTheTonic <gi...@git.apache.org>.
Github user WangTaoTheTonic commented on the pull request:

    https://github.com/apache/spark/pull/4505#issuecomment-142156332
  
    @JoshRosen @andrewor14 @srowen @pfxuan 
    As the network related configurations need to be considered globally, I will close this for now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/4505#issuecomment-73732948
  
      [Test build #27210 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27210/consoleFull) for   PR 4505 at commit [`c71f838`](https://github.com/apache/spark/commit/c71f838fe6ec509aa085d1244df92c7b5cc134aa).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/4505#issuecomment-136918445
  
    @WangTaoTheTonic let's close this patch since I don't think there is sufficient interest in this feature to warrant the additional complexity it introduces.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

Posted by pfxuan <gi...@git.apache.org>.
Github user pfxuan commented on the pull request:

    https://github.com/apache/spark/pull/4505#issuecomment-137005239
  
    @WangTaoTheTonic , this could be a useful feature for further integration with other systems:
    
    http://hortonworks.com/blog/multihoming-on-hadoop-yarn-clusters/
    YARN: https://issues.apache.org/jira/browse/YARN-1994
    Hadoop: https://issues.apache.org/jira/browse/HADOOP-8198
    MapReduce: https://issues.apache.org/jira/browse/MAPREDUCE-4168
    Tachyon: https://github.com/amplab/tachyon/pull/1195


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/4505#issuecomment-73752506
  
      [Test build #27212 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27212/consoleFull) for   PR 4505 at commit [`3a768d4`](https://github.com/apache/spark/commit/3a768d4419745cd7ceb56e3b4648d5de7c35daf4).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class MasterWebUI(val master: Master, hostName: String, requestedPort: Int)`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/4505#issuecomment-73710867
  
      [Test build #27206 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27206/consoleFull) for   PR 4505 at commit [`1442447`](https://github.com/apache/spark/commit/14424471d8332aac6211d0e8e9b9cd11b7d473f4).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class MasterWebUI(val master: Master, hostName: String, requestedPort: Int)`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/4505#discussion_r25010835
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala ---
    @@ -23,46 +23,56 @@ import org.apache.spark.util.{IntParam, MemoryParam, Utils}
     import org.apache.spark.SparkConf
     
     /**
    - * Command-line parser for the worker.
    + * Arguments parser for the worker.
      */
     private[spark] class WorkerArguments(args: Array[String], conf: SparkConf) {
    -  var host = Utils.localHostName()
    -  var port = 0
    -  var webUiPort = 8081
    -  var cores = inferDefaultCores()
    -  var memory = inferDefaultMemory()
    +  var host: String = null
    +  var port: Int = -1
    +  var webUiPort: Int = -1
    +  var cores: Int = -1
    +  var memory: Int = -1
       var masters: Array[String] = null
       var workDir: String = null
       var propertiesFile: String = null
     
    -  // Check for settings in environment variables
    -  if (System.getenv("SPARK_WORKER_PORT") != null) {
    -    port = System.getenv("SPARK_WORKER_PORT").toInt
    -  }
    -  if (System.getenv("SPARK_WORKER_CORES") != null) {
    -    cores = System.getenv("SPARK_WORKER_CORES").toInt
    -  }
    -  if (conf.getenv("SPARK_WORKER_MEMORY") != null) {
    -    memory = Utils.memoryStringToMb(conf.getenv("SPARK_WORKER_MEMORY"))
    -  }
    -  if (System.getenv("SPARK_WORKER_WEBUI_PORT") != null) {
    -    webUiPort = System.getenv("SPARK_WORKER_WEBUI_PORT").toInt
    -  }
    -  if (System.getenv("SPARK_WORKER_DIR") != null) {
    -    workDir = System.getenv("SPARK_WORKER_DIR")
    -  }
    -
       parse(args.toList)
     
       // This mutates the SparkConf, so all accesses to it must be made after this line
       propertiesFile = Utils.loadDefaultSparkProperties(conf, propertiesFile)
     
    -  if (conf.contains("spark.worker.ui.port")) {
    -    webUiPort = conf.get("spark.worker.ui.port").toInt
    -  }
    -
    +  loadEnvironmentArguments()
       checkWorkerMemory()
     
    +  /**
    +   * Load arguments from environment variables, Spark properties etc.
    +   */
    +  private def loadEnvironmentArguments(): Unit = {
    +    host = Option(host)
    +      .orElse(conf.getOption("spark.worker.host"))
    +      .getOrElse(Utils.localHostName())
    +    if (port < 0) {
    +      port = conf.getOption("spark.worker.port").map(_.toInt)
    +        .orElse(Option(conf.getenv("SPARK_WORKER_PORT")).map(_.toInt))
    +        .getOrElse(0)
    --- End diff --
    
    why not just `map(_.toInt)` after the `orElse`? Less duplicate code there


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

Posted by WangTaoTheTonic <gi...@git.apache.org>.
Github user WangTaoTheTonic commented on the pull request:

    https://github.com/apache/spark/pull/4505#issuecomment-73703993
  
    As now we have these configs, is it necessary to keep `SPARK_PUBLIC_DNS`? 
    
    If yes, in which order should we place it?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/4505#issuecomment-73733355
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/27210/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/4505#issuecomment-73752518
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/27212/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/4505#issuecomment-75905956
  
      [Test build #27940 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27940/consoleFull) for   PR 4505 at commit [`5520c70`](https://github.com/apache/spark/commit/5520c70d61db716dd7bbb941e988c3a92b6a01d4).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/4505#issuecomment-73733352
  
      [Test build #27210 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27210/consoleFull) for   PR 4505 at commit [`c71f838`](https://github.com/apache/spark/commit/c71f838fe6ec509aa085d1244df92c7b5cc134aa).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/4505#issuecomment-91971035
  
    @WangTaoTheTonic, to clarify the use-case, is the goal here to _prevent_ requests on certain network interfaces from being able to access Spark components?
    
    This doesn't eliminate the need for SPARK_PUBLIC_DNS, which is used for controlling the advertised externally-routable hostname for things like web UI links, etc.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-5719]allow daemons to bind to specified...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/4505#issuecomment-75911600
  
      [Test build #27940 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27940/consoleFull) for   PR 4505 at commit [`5520c70`](https://github.com/apache/spark/commit/5520c70d61db716dd7bbb941e988c3a92b6a01d4).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class MasterWebUI(val master: Master, hostName: String, requestedPort: Int)`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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