diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala
index d26cccbfe196d8ebcf7f4e29af1e84288972c12b..0afab522af82912407231c481e5367cf5b194c02 100644
--- a/core/src/main/scala/spark/SparkContext.scala
+++ b/core/src/main/scala/spark/SparkContext.scala
@@ -86,7 +86,7 @@ class SparkContext(
 
   // Set Spark master host and port system properties
   if (System.getProperty("spark.master.host") == null) {
-    System.setProperty("spark.master.host", Utils.localIpAddress())
+    System.setProperty("spark.master.host", Utils.localIpAddress)
   }
   if (System.getProperty("spark.master.port") == null) {
     System.setProperty("spark.master.port", "0")
diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala
index c8799e6de3ebc81f5dcff6c268940f2941b3f924..6d64b32174f05f68ea769c39ab691fd4c156a59a 100644
--- a/core/src/main/scala/spark/Utils.scala
+++ b/core/src/main/scala/spark/Utils.scala
@@ -1,12 +1,13 @@
 package spark
 
 import java.io._
-import java.net.{InetAddress, URL, URI}
+import java.net.{NetworkInterface, InetAddress, URL, URI}
 import java.util.{Locale, Random, UUID}
 import java.util.concurrent.{Executors, ThreadFactory, ThreadPoolExecutor}
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.{Path, FileSystem, FileUtil}
 import scala.collection.mutable.ArrayBuffer
+import scala.collection.JavaConversions._
 import scala.io.Source
 
 /**
@@ -199,12 +200,34 @@ private object Utils extends Logging {
   /**
    * Get the local host's IP address in dotted-quad format (e.g. 1.2.3.4).
    */
-  def localIpAddress(): String = {
+  lazy val localIpAddress: String = findLocalIpAddress()
+
+  private def findLocalIpAddress(): String = {
     val defaultIpOverride = System.getenv("SPARK_LOCAL_IP")
-    if (defaultIpOverride != null)
+    if (defaultIpOverride != null) {
       defaultIpOverride
-    else
-      InetAddress.getLocalHost.getHostAddress
+    } else {
+      val address = InetAddress.getLocalHost
+      if (address.isLoopbackAddress) {
+        // Address resolves to something like 127.0.1.1, which happens on Debian; try to find
+        // a better address using the local network interfaces
+        for (ni <- NetworkInterface.getNetworkInterfaces) {
+          for (addr <- ni.getInetAddresses if !addr.isLinkLocalAddress && !addr.isLoopbackAddress) {
+            // We've found an address that looks reasonable!
+            logWarning("Your hostname, " + InetAddress.getLocalHost.getHostName + " resolves to" +
+              " a loopback address: " + address.getHostAddress + "; using " + addr.getHostAddress +
+              " instead (on interface " + ni.getName + ")")
+            logWarning("Set SPARK_LOCAL_IP if you need to bind to another address")
+            return addr.getHostAddress
+          }
+        }
+        logWarning("Your hostname, " + InetAddress.getLocalHost.getHostName + " resolves to" +
+          " a loopback address: " + address.getHostAddress + ", but we couldn't find any" +
+          " external IP address!")
+        logWarning("Set SPARK_LOCAL_IP if you need to bind to another address")
+      }
+      address.getHostAddress
+    }
   }
 
   private var customHostname: Option[String] = None
diff --git a/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala b/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala
index ef27bbb5029c999925d92a8323900e8a1696a896..386f505f2a9db12482c81386bac4414a9c4f4d9e 100644
--- a/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala
+++ b/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala
@@ -48,7 +48,7 @@ private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal:
   // Used only in Workers
   @transient var ttGuide: TalkToGuide = null
 
-  @transient var hostAddress = Utils.localIpAddress()
+  @transient var hostAddress = Utils.localIpAddress
   @transient var listenPort = -1
   @transient var guidePort = -1
 
diff --git a/core/src/main/scala/spark/broadcast/TreeBroadcast.scala b/core/src/main/scala/spark/broadcast/TreeBroadcast.scala
index fa676e90648f27b9f50a1491a37e10507fe40288..f573512835d03ce5f807847fa8d3735a2927046d 100644
--- a/core/src/main/scala/spark/broadcast/TreeBroadcast.scala
+++ b/core/src/main/scala/spark/broadcast/TreeBroadcast.scala
@@ -36,7 +36,7 @@ extends Broadcast[T](id) with Logging with Serializable {
   @transient var serveMR: ServeMultipleRequests = null
   @transient var guideMR: GuideMultipleRequests = null
 
-  @transient var hostAddress = Utils.localIpAddress()
+  @transient var hostAddress = Utils.localIpAddress
   @transient var listenPort = -1
   @transient var guidePort = -1
 
@@ -138,7 +138,7 @@ extends Broadcast[T](id) with Logging with Serializable {
 
     serveMR =  null
 
-    hostAddress = Utils.localIpAddress()
+    hostAddress = Utils.localIpAddress
     listenPort = -1
 
     stopBroadcast = false
diff --git a/core/src/main/scala/spark/deploy/client/TestClient.scala b/core/src/main/scala/spark/deploy/client/TestClient.scala
index bf0e7428bac3c1f8cb9cf83e4e2ba5c6bf038ea5..5b710f5520a8658c4e4fea1e05cd3a5a62e275d0 100644
--- a/core/src/main/scala/spark/deploy/client/TestClient.scala
+++ b/core/src/main/scala/spark/deploy/client/TestClient.scala
@@ -23,7 +23,7 @@ private[spark] object TestClient {
 
   def main(args: Array[String]) {
     val url = args(0)
-    val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress(), 0)
+    val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0)
     val desc = new JobDescription(
       "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()))
     val listener = new TestListener