diff --git a/.rat-excludes b/.rat-excludes
index 85bfad60fcadc3ef10c458874c118e7985993802..a2b5665a0be26b7040637e8dcb7cd9566d5c7ad8 100644
--- a/.rat-excludes
+++ b/.rat-excludes
@@ -39,4 +39,4 @@ work
 .*\.q
 golden
 test.out/*
-.*iml
+.*iml
\ No newline at end of file
diff --git a/core/src/test/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala
similarity index 84%
rename from core/src/test/scala/org/apache/spark/TestUtils.scala
rename to core/src/main/scala/org/apache/spark/TestUtils.scala
index 1611d09652d409edfcc5d8abe4f910e94864ea3a..4597595a838e3ea0b7b8a40cf8a2d14d6d5bab1b 100644
--- a/core/src/test/scala/org/apache/spark/TestUtils.scala
+++ b/core/src/main/scala/org/apache/spark/TestUtils.scala
@@ -26,7 +26,14 @@ import scala.collection.JavaConversions._
 import javax.tools.{JavaFileObject, SimpleJavaFileObject, ToolProvider}
 import com.google.common.io.Files
 
-object TestUtils {
+/**
+ * Utilities for tests. Included in main codebase since it's used by multiple
+ * projects.
+ *
+ * TODO: See if we can move this to the test codebase by specifying
+ * test dependencies between projects. 
+ */
+private[spark] object TestUtils {
 
   /**
    * Create a jar that defines classes with the given names.
@@ -34,13 +41,14 @@ object TestUtils {
    * Note: if this is used during class loader tests, class names should be unique
    * in order to avoid interference between tests.
    */
-  def createJarWithClasses(classNames: Seq[String]): URL = {
+  def createJarWithClasses(classNames: Seq[String], value: String = ""): URL = {
     val tempDir = Files.createTempDir()
-    val files = for (name <- classNames) yield createCompiledClass(name, tempDir)
+    val files = for (name <- classNames) yield createCompiledClass(name, tempDir, value)
     val jarFile = new File(tempDir, "testJar-%s.jar".format(System.currentTimeMillis()))
     createJar(files, jarFile)
   }
 
+
   /**
    * Create a jar file that contains this set of files. All files will be located at the root
    * of the jar.
@@ -80,9 +88,11 @@ object TestUtils {
   }
 
   /** Creates a compiled class with the given name. Class file will be placed in destDir. */
-  def createCompiledClass(className: String, destDir: File): File = {
+  def createCompiledClass(className: String, destDir: File, value: String = ""): File = {
     val compiler = ToolProvider.getSystemJavaCompiler
-    val sourceFile = new JavaSourceFromString(className, s"public class $className {}")
+    val sourceFile = new JavaSourceFromString(className,
+      "public class " + className + " { @Override public String toString() { " +
+       "return \"" + value + "\";}}")
 
     // Calling this outputs a class file in pwd. It's easier to just rename the file than
     // build a custom FileManager that controls the output location.
diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala
index aecb069e4202b06daf71be72acbef86d6c014c66..c12bd922d40e47066487facb2a163266e4d68115 100644
--- a/core/src/main/scala/org/apache/spark/executor/Executor.scala
+++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala
@@ -291,15 +291,19 @@ private[spark] class Executor(
    * Create a ClassLoader for use in tasks, adding any JARs specified by the user or any classes
    * created by the interpreter to the search path
    */
-  private def createClassLoader(): ExecutorURLClassLoader = {
-    val loader = Thread.currentThread().getContextClassLoader
+  private def createClassLoader(): MutableURLClassLoader = {
+    val loader = this.getClass.getClassLoader
 
     // For each of the jars in the jarSet, add them to the class loader.
     // We assume each of the files has already been fetched.
     val urls = currentJars.keySet.map { uri =>
       new File(uri.split("/").last).toURI.toURL
     }.toArray
-    new ExecutorURLClassLoader(urls, loader)
+    val userClassPathFirst = conf.getBoolean("spark.files.userClassPathFirst", false)
+    userClassPathFirst match {
+      case true => new ChildExecutorURLClassLoader(urls, loader)
+      case false => new ExecutorURLClassLoader(urls, loader)
+    }
   }
 
   /**
@@ -310,11 +314,14 @@ private[spark] class Executor(
     val classUri = conf.get("spark.repl.class.uri", null)
     if (classUri != null) {
       logInfo("Using REPL class URI: " + classUri)
+      val userClassPathFirst: java.lang.Boolean =
+        conf.getBoolean("spark.files.userClassPathFirst", false)
       try {
         val klass = Class.forName("org.apache.spark.repl.ExecutorClassLoader")
           .asInstanceOf[Class[_ <: ClassLoader]]
-        val constructor = klass.getConstructor(classOf[String], classOf[ClassLoader])
-        constructor.newInstance(classUri, parent)
+        val constructor = klass.getConstructor(classOf[String], classOf[ClassLoader],
+          classOf[Boolean])
+        constructor.newInstance(classUri, parent, userClassPathFirst)
       } catch {
         case _: ClassNotFoundException =>
           logError("Could not find org.apache.spark.repl.ExecutorClassLoader on classpath!")
diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala
index f9bfe8ed2f5baa2a60c94c3b6d765bcefa3bd557..208e77073fd03eee6b6b333c7f34c1b459e64803 100644
--- a/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala
+++ b/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala
@@ -19,13 +19,56 @@ package org.apache.spark.executor
 
 import java.net.{URLClassLoader, URL}
 
+import org.apache.spark.util.ParentClassLoader
+
 /**
  * The addURL method in URLClassLoader is protected. We subclass it to make this accessible.
+ * We also make changes so user classes can come before the default classes.
  */
+
+private[spark] trait MutableURLClassLoader extends ClassLoader {
+  def addURL(url: URL)
+  def getURLs: Array[URL]
+}
+
+private[spark] class ChildExecutorURLClassLoader(urls: Array[URL], parent: ClassLoader)
+  extends MutableURLClassLoader {
+
+  private object userClassLoader extends URLClassLoader(urls, null){
+    override def addURL(url: URL) {
+      super.addURL(url)
+    }
+    override def findClass(name: String): Class[_] = { 
+      super.findClass(name)
+    }
+  }
+
+  private val parentClassLoader = new ParentClassLoader(parent)
+
+  override def findClass(name: String): Class[_] = {
+    try {
+      userClassLoader.findClass(name)
+    } catch {
+      case e: ClassNotFoundException => {
+        parentClassLoader.loadClass(name)
+      }
+    }
+  }
+
+  def addURL(url: URL) {
+    userClassLoader.addURL(url)
+  }
+
+  def getURLs() = {
+    userClassLoader.getURLs()
+  }
+}
+
 private[spark] class ExecutorURLClassLoader(urls: Array[URL], parent: ClassLoader)
-  extends URLClassLoader(urls, parent) {
+  extends URLClassLoader(urls, parent) with MutableURLClassLoader {
 
   override def addURL(url: URL) {
     super.addURL(url)
   }
 }
+
diff --git a/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala b/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala
new file mode 100644
index 0000000000000000000000000000000000000000..3abc12681fe9a5a2d7669752e4d604a6bf8b81f5
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala
@@ -0,0 +1,32 @@
+/*
+ * 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.util
+
+/**
+ * A class loader which makes findClass accesible to the child
+ */
+private[spark] class ParentClassLoader(parent: ClassLoader) extends ClassLoader(parent) {
+
+  override def findClass(name: String) = {
+    super.findClass(name)
+  }
+
+  override def loadClass(name: String): Class[_] = {
+    super.loadClass(name)
+  }
+}
diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..c40cfc0696fce2bf695372b9ca73178e3021da70
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala
@@ -0,0 +1,67 @@
+/*
+ * 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.executor
+
+import java.io.File
+import java.net.URLClassLoader
+
+import org.scalatest.FunSuite
+
+import org.apache.spark.TestUtils
+
+class ExecutorURLClassLoaderSuite extends FunSuite {
+
+  val childClassNames = List("FakeClass1", "FakeClass2")
+  val parentClassNames = List("FakeClass1", "FakeClass2", "FakeClass3")
+  val urls = List(TestUtils.createJarWithClasses(childClassNames, "1")).toArray
+  val urls2 = List(TestUtils.createJarWithClasses(parentClassNames, "2")).toArray
+
+  test("child first") {
+    val parentLoader = new URLClassLoader(urls2, null)
+    val classLoader = new ChildExecutorURLClassLoader(urls, parentLoader)
+    val fakeClass = classLoader.loadClass("FakeClass2").newInstance()
+    val fakeClassVersion = fakeClass.toString
+    assert(fakeClassVersion === "1")
+  }
+
+  test("parent first") {
+    val parentLoader = new URLClassLoader(urls2, null)
+    val classLoader = new ExecutorURLClassLoader(urls, parentLoader)
+    val fakeClass = classLoader.loadClass("FakeClass1").newInstance()
+    val fakeClassVersion = fakeClass.toString
+    assert(fakeClassVersion === "2")
+  }
+
+  test("child first can fall back") {
+    val parentLoader = new URLClassLoader(urls2, null)
+    val classLoader = new ChildExecutorURLClassLoader(urls, parentLoader)
+    val fakeClass = classLoader.loadClass("FakeClass3").newInstance()
+    val fakeClassVersion = fakeClass.toString
+    assert(fakeClassVersion === "2")
+  }
+
+  test("child first can fail") {
+    val parentLoader = new URLClassLoader(urls2, null)
+    val classLoader = new ChildExecutorURLClassLoader(urls, parentLoader)
+    intercept[java.lang.ClassNotFoundException] {
+      classLoader.loadClass("FakeClassDoesNotExist").newInstance()
+    }
+  }
+
+
+}
diff --git a/docs/configuration.md b/docs/configuration.md
index 57bda20edcdf1910c3ca4e845982a0d8188d58d7..9c602402f063560819469ad6b30c41598800a5cf 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -596,6 +596,15 @@ Apart from these, the following properties are also available, and may be useful
     the driver.
   </td>
 </tr>
+<tr>
+  <td>spark.files.userClassPathFirst</td>
+  <td>false</td>
+  <td>
+    (Experimental) Whether to give user-added jars precedence over Spark's own jars when
+    loading classes in Executors. This feature can be used to mitigate conflicts between
+    Spark's dependencies and user dependencies. It is currently an experimental feature.
+  </td>
+</tr>
 <tr>
   <td>spark.authenticate</td>
   <td>false</td>
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 08667aac2cd2da364bc77055729a6b0377530802..694f90a83ab67e58e0702bb6d9f556909cce1309 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -195,6 +195,7 @@ object SparkBuild extends Build {
     concurrentRestrictions in Global += Tags.limit(Tags.Test, 1),
 
     resolvers ++= Seq(
+      // HTTPS is unavailable for Maven Central
       "Maven Repository"     at "http://repo.maven.apache.org/maven2",
       "Apache Repository"    at "https://repository.apache.org/content/repositories/releases",
       "JBoss Repository"     at "https://repository.jboss.org/nexus/content/repositories/releases/",
diff --git a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala
index bf73800388ebfb18d0426d7056907494484662be..a30dcfdcecf27848f63bba3aed5315eb2d1037ff 100644
--- a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala
@@ -26,21 +26,23 @@ import org.apache.hadoop.fs.{FileSystem, Path}
 
 import org.apache.spark.SparkEnv
 import org.apache.spark.util.Utils
-
+import org.apache.spark.util.ParentClassLoader
 
 import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm._
 import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._
 
-
 /**
  * A ClassLoader that reads classes from a Hadoop FileSystem or HTTP URI,
- * used to load classes defined by the interpreter when the REPL is used
- */
-class ExecutorClassLoader(classUri: String, parent: ClassLoader)
-extends ClassLoader(parent) {
+ * used to load classes defined by the interpreter when the REPL is used.
+ * Allows the user to specify if user class path should be first
+ */ 
+class ExecutorClassLoader(classUri: String, parent: ClassLoader,
+    userClassPathFirst: Boolean) extends ClassLoader {
   val uri = new URI(classUri)
   val directory = uri.getPath
 
+  val parentLoader = new ParentClassLoader(parent)
+
   // Hadoop FileSystem object for our URI, if it isn't using HTTP
   var fileSystem: FileSystem = {
     if (uri.getScheme() == "http") {
@@ -49,8 +51,27 @@ extends ClassLoader(parent) {
       FileSystem.get(uri, new Configuration())
     }
   }
-  
+
   override def findClass(name: String): Class[_] = {
+    userClassPathFirst match {
+      case true => findClassLocally(name).getOrElse(parentLoader.loadClass(name))
+      case false => {
+        try {
+          parentLoader.loadClass(name)
+        } catch {
+          case e: ClassNotFoundException => {
+            val classOption = findClassLocally(name)
+            classOption match {
+              case None => throw new ClassNotFoundException(name, e)
+              case Some(a) => a
+            }
+          }
+        }
+      }
+    }
+  }
+
+  def findClassLocally(name: String): Option[Class[_]] = {
     try {
       val pathInDirectory = name.replace('.', '/') + ".class"
       val inputStream = {
@@ -68,9 +89,9 @@ extends ClassLoader(parent) {
       }
       val bytes = readAndTransformClass(name, inputStream)
       inputStream.close()
-      return defineClass(name, bytes, 0, bytes.length)
+      Some(defineClass(name, bytes, 0, bytes.length))
     } catch {
-      case e: Exception => throw new ClassNotFoundException(name, e)
+      case e: Exception => None
     }
   }
   
diff --git a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..336df988a1b7f8a36469397cad49733029ad9202
--- /dev/null
+++ b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala
@@ -0,0 +1,76 @@
+/*
+ * 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.repl
+
+import java.io.File
+import java.net.URLClassLoader
+
+import org.scalatest.BeforeAndAfterAll
+import org.scalatest.FunSuite
+
+import com.google.common.io.Files
+
+import org.apache.spark.TestUtils
+
+class ExecutorClassLoaderSuite extends FunSuite with BeforeAndAfterAll {
+
+  val childClassNames = List("ReplFakeClass1", "ReplFakeClass2")
+  val parentClassNames = List("ReplFakeClass1", "ReplFakeClass2", "ReplFakeClass3")
+  val tempDir1 = Files.createTempDir()
+  val tempDir2 = Files.createTempDir()
+  val url1 = "file://" + tempDir1
+  val urls2 = List(tempDir2.toURI.toURL).toArray
+
+  override def beforeAll() {
+    childClassNames.foreach(TestUtils.createCompiledClass(_, tempDir1, "1"))
+    parentClassNames.foreach(TestUtils.createCompiledClass(_, tempDir2, "2"))
+  }
+
+  test("child first") {
+    val parentLoader = new URLClassLoader(urls2, null)
+    val classLoader = new ExecutorClassLoader(url1, parentLoader, true)
+    val fakeClass = classLoader.loadClass("ReplFakeClass2").newInstance()
+    val fakeClassVersion = fakeClass.toString
+    assert(fakeClassVersion === "1")
+  }
+
+  test("parent first") {
+    val parentLoader = new URLClassLoader(urls2, null)
+    val classLoader = new ExecutorClassLoader(url1, parentLoader, false)
+    val fakeClass = classLoader.loadClass("ReplFakeClass1").newInstance()
+    val fakeClassVersion = fakeClass.toString
+    assert(fakeClassVersion === "2")
+  }
+
+  test("child first can fall back") {
+    val parentLoader = new URLClassLoader(urls2, null)
+    val classLoader = new ExecutorClassLoader(url1, parentLoader, true)
+    val fakeClass = classLoader.loadClass("ReplFakeClass3").newInstance()
+    val fakeClassVersion = fakeClass.toString
+    assert(fakeClassVersion === "2")
+  }
+
+  test("child first can fail") {
+    val parentLoader = new URLClassLoader(urls2, null)
+    val classLoader = new ExecutorClassLoader(url1, parentLoader, true)
+    intercept[java.lang.ClassNotFoundException] {
+      classLoader.loadClass("ReplFakeClassDoesNotExist").newInstance()
+    }
+  }
+
+}