You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by aarondav <gi...@git.apache.org> on 2015/04/18 19:56:59 UTC

[GitHub] spark pull request: [SPARK-6955] Perform port retries at NettyBloc...

GitHub user aarondav opened a pull request:

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

    [SPARK-6955] Perform port retries at NettyBlockTransferService level

    Currently we're doing port retries in the TransportServer level, but this is not specified by the TransportContext API and it has other further-reaching impacts like causing undesirable behavior for the Yarn and Standalone shuffle services.

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

    $ git pull https://github.com/aarondav/spark port-bind

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

    https://github.com/apache/spark/pull/5575.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 #5575
    
----
commit c671f6fe28a65acc18034785dc5566ac475c31f0
Author: Aaron Davidson <aa...@databricks.com>
Date:   2015-04-18T17:46:05Z

    [SPARK-6955] Perform port retries at NettyBlockTransferService level
    
    Currently we're doing port retries in the TransportServer level, but this is not specified by the TransportContext API and it has other further-reaching impacts like causing undesirable behvior for the Yarn and Standalone shuffle services.

----


---
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-6955] Perform port retries at NettyBloc...

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

    https://github.com/apache/spark/pull/5575#issuecomment-94318343
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/30568/
    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-6955] Perform port retries at NettyBloc...

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

    https://github.com/apache/spark/pull/5575#issuecomment-94329380
  
      [Test build #30569 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30569/consoleFull) for   PR 5575 at commit [`df60b80`](https://github.com/apache/spark/commit/df60b804cf782d67ca013a24bdf95ffa06a7c13e).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.
     * This patch does not change any dependencies.


---
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-6955] Perform port retries at NettyBloc...

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

    https://github.com/apache/spark/pull/5575#issuecomment-94321419
  
      [Test build #30569 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30569/consoleFull) for   PR 5575 at commit [`df60b80`](https://github.com/apache/spark/commit/df60b804cf782d67ca013a24bdf95ffa06a7c13e).


---
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-6955] Perform port retries at NettyBloc...

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

    https://github.com/apache/spark/pull/5575#issuecomment-94318339
  
    **[Test build #30568 timed out](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30568/consoleFull)**     for PR 5575 at commit [`a3dc42d`](https://github.com/apache/spark/commit/a3dc42d00fba346ed479cdf5c0c59512aa60aafc)     after a configured wait of `150m`.


---
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-6955] Perform port retries at NettyBloc...

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

    https://github.com/apache/spark/pull/5575#discussion_r28740458
  
    --- Diff: core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala ---
    @@ -43,6 +43,9 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll {
         conf.set("spark.shuffle.manager", "sort")
         conf.set("spark.shuffle.service.enabled", "true")
         conf.set("spark.shuffle.service.port", server.getPort.toString)
    +
    +    // local-cluster mode starts a Worker which would start its own shuffle service without this:
    +    conf.set("spark.worker.shouldHostShuffleServiceIfEnabled", "false")
    --- End diff --
    
    Do we actually ever want to start an external shuffle service in a local cluster? If not I think it makes more sense to just set `spark.shuffle.service.enabled` to false in `LocalSparkCluster` (we already do this for the REST submission server for Master)


---
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-6955] Perform port retries at NettyBloc...

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

    https://github.com/apache/spark/pull/5575#discussion_r28711548
  
    --- Diff: core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala ---
    @@ -0,0 +1,78 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.spark.network.netty
    +
    +import org.apache.spark.network.BlockDataManager
    +import org.apache.spark.{SecurityManager, SparkConf}
    +import org.mockito.Mockito.mock
    +import org.scalatest._
    +
    +class NettyBlockTransferServiceSuite extends FunSuite with BeforeAndAfterEach with ShouldMatchers {
    +  private var service0: NettyBlockTransferService = _
    +  private var service1: NettyBlockTransferService = _
    +
    +  override def afterEach() {
    +    if (service0 != null) {
    +      service0.close()
    +      service0 = null
    +    }
    +
    +    if (service1 != null) {
    +      service1.close()
    +      service1 = null
    +    }
    +  }
    +
    +  test("can bind to a random port") {
    +    service0 = createService(port = 0)
    +    service0.port should not be 0
    +  }
    +
    +  test("can bind to two random ports") {
    +    service0 = createService(port = 0)
    +    service1 = createService(port = 0)
    +    service0.port should not be service1.port
    +  }
    +
    +  test("can bind to a specific port") {
    +    val port = 17634
    +    service0 = createService(port)
    +    service0.port should be >= port
    +    service0.port should be <= (port + 10) // avoid testing equality in case of simultaneous tests
    --- End diff --
    
    This seems flaky. Isn't the previous check enough? Otherwise, instead of `10` I'd use `1024` which is the actual limit in the code.


---
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-6955] Perform port retries at NettyBloc...

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

    https://github.com/apache/spark/pull/5575#discussion_r28721339
  
    --- Diff: core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala ---
    @@ -0,0 +1,78 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.spark.network.netty
    +
    +import org.apache.spark.network.BlockDataManager
    +import org.apache.spark.{SecurityManager, SparkConf}
    +import org.mockito.Mockito.mock
    +import org.scalatest._
    +
    +class NettyBlockTransferServiceSuite extends FunSuite with BeforeAndAfterEach with ShouldMatchers {
    +  private var service0: NettyBlockTransferService = _
    +  private var service1: NettyBlockTransferService = _
    +
    +  override def afterEach() {
    +    if (service0 != null) {
    +      service0.close()
    +      service0 = null
    +    }
    +
    +    if (service1 != null) {
    +      service1.close()
    +      service1 = null
    +    }
    +  }
    +
    +  test("can bind to a random port") {
    +    service0 = createService(port = 0)
    +    service0.port should not be 0
    +  }
    +
    +  test("can bind to two random ports") {
    +    service0 = createService(port = 0)
    +    service1 = createService(port = 0)
    +    service0.port should not be service1.port
    +  }
    +
    +  test("can bind to a specific port") {
    +    val port = 17634
    +    service0 = createService(port)
    +    service0.port should be >= port
    +    service0.port should be <= (port + 10) // avoid testing equality in case of simultaneous tests
    --- End diff --
    
    I can increase the number no problem, though I guess we should use `Utils.portMaxRetries` instead. However, note that even 10 should be very unlikely to be hit, that would mean there's 5-10 concurrently running tests at this exact part of the build, or else we're very, very unlucky.


---
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-6955] Perform port retries at NettyBloc...

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

    https://github.com/apache/spark/pull/5575#issuecomment-94307038
  
    Jenkins, retest this please.


---
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-6955] Perform port retries at NettyBloc...

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

    https://github.com/apache/spark/pull/5575#issuecomment-94187639
  
      [Test build #30526 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30526/consoleFull) for   PR 5575 at commit [`c671f6f`](https://github.com/apache/spark/commit/c671f6fe28a65acc18034785dc5566ac475c31f0).
     * This patch **fails RAT tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.
     * This patch does not change any dependencies.


---
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-6955] Perform port retries at NettyBloc...

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

    https://github.com/apache/spark/pull/5575#issuecomment-94199113
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/30527/
    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-6955] Perform port retries at NettyBloc...

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

    https://github.com/apache/spark/pull/5575#issuecomment-94187640
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/30526/
    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-6955] Perform port retries at NettyBloc...

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

    https://github.com/apache/spark/pull/5575#issuecomment-94187636
  
      [Test build #30526 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30526/consoleFull) for   PR 5575 at commit [`c671f6f`](https://github.com/apache/spark/commit/c671f6fe28a65acc18034785dc5566ac475c31f0).


---
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-6955] Perform port retries at NettyBloc...

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

    https://github.com/apache/spark/pull/5575#discussion_r28740515
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/worker/StandaloneWorkerShuffleService.scala ---
    @@ -35,7 +35,10 @@ private[worker]
     class StandaloneWorkerShuffleService(sparkConf: SparkConf, securityManager: SecurityManager)
       extends Logging {
     
    -  private val enabled = sparkConf.getBoolean("spark.shuffle.service.enabled", false)
    +  // Check both if shuffle service is enabled, and that the worker should actually host the
    +  // shuffle service in that case. (The latter is currently only used for testing.)
    +  private val enabled = sparkConf.getBoolean("spark.shuffle.service.enabled", false) &&
    +    sparkConf.getBoolean("spark.worker.shouldHostShuffleServiceIfEnabled", true)
    --- End diff --
    
    FYI this change will conflict with https://github.com/apache/spark/pull/4990, which makes this a general class that Mesos can also use. If you follow my suggestion of just disabling this for local cluster then this doesn't need to change.


---
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-6955] Perform port retries at NettyBloc...

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

    https://github.com/apache/spark/pull/5575#issuecomment-94243393
  
    Jenkins, retest this please.


---
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-6955] Perform port retries at NettyBloc...

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

    https://github.com/apache/spark/pull/5575#issuecomment-94199111
  
    **[Test build #30527 timed out](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30527/consoleFull)**     for PR 5575 at commit [`a3dc42d`](https://github.com/apache/spark/commit/a3dc42d00fba346ed479cdf5c0c59512aa60aafc)     after a configured wait of `150m`.


---
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-6955] Perform port retries at NettyBloc...

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

    https://github.com/apache/spark/pull/5575#issuecomment-94243620
  
      [Test build #30548 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30548/consoleFull) for   PR 5575 at commit [`a3dc42d`](https://github.com/apache/spark/commit/a3dc42d00fba346ed479cdf5c0c59512aa60aafc).


---
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-6955] Perform port retries at NettyBloc...

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

    https://github.com/apache/spark/pull/5575#issuecomment-94329386
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/30569/
    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-6955] Perform port retries at NettyBloc...

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

    https://github.com/apache/spark/pull/5575#issuecomment-94187977
  
    cc @SaintBacchus @andrewor13 @vanzin This patch aims to resolve SPARK-6955 while keeping the fix for SPARK-5444.


---
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-6955] Perform port retries at NettyBloc...

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

    https://github.com/apache/spark/pull/5575#issuecomment-94188022
  
      [Test build #30527 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30527/consoleFull) for   PR 5575 at commit [`a3dc42d`](https://github.com/apache/spark/commit/a3dc42d00fba346ed479cdf5c0c59512aa60aafc).


---
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-6955] Perform port retries at NettyBloc...

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

    https://github.com/apache/spark/pull/5575#issuecomment-96767480
  
      [Test build #31002 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/31002/consoleFull) for   PR 5575 at commit [`df60b80`](https://github.com/apache/spark/commit/df60b804cf782d67ca013a24bdf95ffa06a7c13e).


---
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-6955] Perform port retries at NettyBloc...

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

    https://github.com/apache/spark/pull/5575#issuecomment-94252172
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/30548/
    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-6955] Perform port retries at NettyBloc...

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

    https://github.com/apache/spark/pull/5575#issuecomment-94252170
  
    **[Test build #30548 timed out](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30548/consoleFull)**     for PR 5575 at commit [`a3dc42d`](https://github.com/apache/spark/commit/a3dc42d00fba346ed479cdf5c0c59512aa60aafc)     after a configured wait of `150m`.


---
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-6955] Perform port retries at NettyBloc...

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

    https://github.com/apache/spark/pull/5575#issuecomment-94307271
  
      [Test build #30568 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30568/consoleFull) for   PR 5575 at commit [`a3dc42d`](https://github.com/apache/spark/commit/a3dc42d00fba346ed479cdf5c0c59512aa60aafc).


---
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-6955] Perform port retries at NettyBloc...

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

    https://github.com/apache/spark/pull/5575#issuecomment-94188006
  
    err, cc @andrewor14, not @andrewor13.


---
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-6955] Perform port retries at NettyBloc...

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

    https://github.com/apache/spark/pull/5575#issuecomment-94523299
  
    LGTM aside from the potentially flaky test.


---
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-6955] Perform port retries at NettyBloc...

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

    https://github.com/apache/spark/pull/5575#issuecomment-94338939
  
    I seems to be a better way.


---
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