From bb8bd11da51b3b4b59b921d9d2a550c78a865ee5 Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Tue, 13 Jan 2015 09:28:21 -0800 Subject: [SPARK-5006][Deploy]spark.port.maxRetries doesn't work https://issues.apache.org/jira/browse/SPARK-5006 I think the issue is produced in https://github.com/apache/spark/pull/1777. Not digging mesos's backend yet. Maybe should add same logic either. Author: WangTaoTheTonic Author: WangTao Closes #3841 from WangTaoTheTonic/SPARK-5006 and squashes the following commits: 8cdf96d [WangTao] indent thing 2d86d65 [WangTaoTheTonic] fix line length 7cdfd98 [WangTaoTheTonic] fit for new HttpServer constructor 61a370d [WangTaoTheTonic] some minor fixes bc6e1ec [WangTaoTheTonic] rebase 67bcb46 [WangTaoTheTonic] put conf at 3rd position, modify suite class, add comments f450cd1 [WangTaoTheTonic] startServiceOnPort will use a SparkConf arg 29b751b [WangTaoTheTonic] rebase as ExecutorRunnableUtil changed to ExecutorRunnable 396c226 [WangTaoTheTonic] make the grammar more like scala 191face [WangTaoTheTonic] invalid value name 62ec336 [WangTaoTheTonic] spark.port.maxRetries doesn't work Conflicts: external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala --- .../scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala | 2 +- .../test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) (limited to 'external') diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala index 13943ed544..f333e3891b 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala @@ -80,7 +80,7 @@ class FlumeStreamSuite extends FunSuite with BeforeAndAfter with Matchers with L val socket = new ServerSocket(trialPort) socket.close() (null, trialPort) - })._2 + }, conf)._2 } /** Setup and start the streaming context */ diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala index 98fe6cb301..e816255aef 100644 --- a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala +++ b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.streaming.mqtt import java.net.{URI, ServerSocket} import org.apache.activemq.broker.{TransportConnector, BrokerService} -import org.apache.spark.util.Utils import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.concurrent.Eventually import scala.concurrent.duration._ @@ -29,6 +28,8 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.ReceiverInputDStream import org.eclipse.paho.client.mqttv3._ import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence +import org.apache.spark.SparkConf +import org.apache.spark.util.Utils class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter { @@ -101,7 +102,7 @@ class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter { val socket = new ServerSocket(trialPort) socket.close() (null, trialPort) - })._2 + }, new SparkConf())._2 } def publishData(data: String): Unit = { -- cgit v1.2.3