diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala
index d9a671687aad0e19b1552f9e96e2633119658984..0b5a914e7dbbfc089583e74827ff188ec75344c0 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -1876,6 +1876,10 @@ private[spark] object Utils extends Logging {
       startService: Int => (T, Int),
       conf: SparkConf,
       serviceName: String = ""): (T, Int) = {
+
+    require(startPort == 0 || (1024 <= startPort && startPort < 65536),
+      "startPort should be between 1024 and 65535 (inclusive), or 0 for a random free port.")
+
     val serviceString = if (serviceName.isEmpty) "" else s" '$serviceName'"
     val maxRetries = portMaxRetries(conf)
     for (offset <- 0 to maxRetries) {
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 322de7bf2fed814ef0ad190f7b82ebee148fbf47..51d273af8da848cbca52df3a6120511d7c8f91d7 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
@@ -28,6 +28,7 @@ import scala.language.postfixOps
 import com.google.common.base.Charsets
 import org.apache.avro.ipc.NettyTransceiver
 import org.apache.avro.ipc.specific.SpecificRequestor
+import org.apache.commons.lang3.RandomUtils
 import org.apache.flume.source.avro
 import org.apache.flume.source.avro.{AvroFlumeEvent, AvroSourceProtocol}
 import org.jboss.netty.channel.ChannelPipeline
@@ -40,7 +41,6 @@ import org.scalatest.concurrent.Eventually._
 import org.apache.spark.{Logging, SparkConf}
 import org.apache.spark.storage.StorageLevel
 import org.apache.spark.streaming.{Milliseconds, StreamingContext, TestOutputStream}
-import org.apache.spark.streaming.scheduler.{StreamingListener, StreamingListenerReceiverStarted}
 import org.apache.spark.util.Utils
 
 class FlumeStreamSuite extends FunSuite with BeforeAndAfter with Matchers with Logging {
@@ -76,7 +76,8 @@ class FlumeStreamSuite extends FunSuite with BeforeAndAfter with Matchers with L
 
   /** Find a free port */
   private def findFreePort(): Int = {
-    Utils.startServiceOnPort(23456, (trialPort: Int) => {
+    val candidatePort = RandomUtils.nextInt(1024, 65536)
+    Utils.startServiceOnPort(candidatePort, (trialPort: Int) => {
       val socket = new ServerSocket(trialPort)
       socket.close()
       (null, trialPort)
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 0f3298af6234a1d185053f46e636f64a1b0e1f1e..24d78ecb3a97d980cc85e3ed273c218e5a8b3d94 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
@@ -25,6 +25,7 @@ import scala.concurrent.duration._
 import scala.language.postfixOps
 
 import org.apache.activemq.broker.{TransportConnector, BrokerService}
+import org.apache.commons.lang3.RandomUtils
 import org.eclipse.paho.client.mqttv3._
 import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence
 
@@ -113,7 +114,8 @@ class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter {
   }
 
   private def findFreePort(): Int = {
-    Utils.startServiceOnPort(23456, (trialPort: Int) => {
+    val candidatePort = RandomUtils.nextInt(1024, 65536)
+    Utils.startServiceOnPort(candidatePort, (trialPort: Int) => {
       val socket = new ServerSocket(trialPort)
       socket.close()
       (null, trialPort)