diff --git a/core/src/main/scala/spark/BoundedMemoryCache.scala b/core/src/main/scala/spark/BoundedMemoryCache.scala
deleted file mode 100644
index e8392a194f139fa97578bf037163ccb9d12a559e..0000000000000000000000000000000000000000
--- a/core/src/main/scala/spark/BoundedMemoryCache.scala
+++ /dev/null
@@ -1,118 +0,0 @@
-package spark
-
-import java.util.LinkedHashMap
-
-/**
- * An implementation of Cache that estimates the sizes of its entries and attempts to limit its
- * total memory usage to a fraction of the JVM heap. Objects' sizes are estimated using
- * SizeEstimator, which has limitations; most notably, we will overestimate total memory used if
- * some cache entries have pointers to a shared object. Nonetheless, this Cache should work well
- * when most of the space is used by arrays of primitives or of simple classes.
- */
-private[spark] class BoundedMemoryCache(maxBytes: Long) extends Cache with Logging {
-  logInfo("BoundedMemoryCache.maxBytes = " + maxBytes)
-
-  def this() {
-    this(BoundedMemoryCache.getMaxBytes)
-  }
-
-  private var currentBytes = 0L
-  private val map = new LinkedHashMap[(Any, Int), Entry](32, 0.75f, true)
-
-  override def get(datasetId: Any, partition: Int): Any = {
-    synchronized {
-      val entry = map.get((datasetId, partition))
-      if (entry != null) {
-        entry.value
-      } else {
-        null
-      }
-    }
-  }
-
-  override def put(datasetId: Any, partition: Int, value: Any): CachePutResponse = {
-    val key = (datasetId, partition)
-    logInfo("Asked to add key " + key)
-    val size = estimateValueSize(key, value)
-    synchronized {
-      if (size > getCapacity) {
-        return CachePutFailure()
-      } else if (ensureFreeSpace(datasetId, size)) {
-        logInfo("Adding key " + key)
-        map.put(key, new Entry(value, size))
-        currentBytes += size
-        logInfo("Number of entries is now " + map.size)
-        return CachePutSuccess(size)
-      } else {
-        logInfo("Didn't add key " + key + " because we would have evicted part of same dataset")
-        return CachePutFailure()
-      }
-    }
-  }
-
-  override def getCapacity: Long = maxBytes
-
-  /**
-   * Estimate sizeOf 'value'
-   */
-  private def estimateValueSize(key: (Any, Int), value: Any) = {
-    val startTime = System.currentTimeMillis
-    val size = SizeEstimator.estimate(value.asInstanceOf[AnyRef])
-    val timeTaken = System.currentTimeMillis - startTime
-    logInfo("Estimated size for key %s is %d".format(key, size))
-    logInfo("Size estimation for key %s took %d ms".format(key, timeTaken))
-    size
-  }
-
-  /**
-   * Remove least recently used entries from the map until at least space bytes are free, in order
-   * to make space for a partition from the given dataset ID. If this cannot be done without
-   * evicting other data from the same dataset, returns false; otherwise, returns true. Assumes
-   * that a lock is held on the BoundedMemoryCache.
-   */
-  private def ensureFreeSpace(datasetId: Any, space: Long): Boolean = {
-    logInfo("ensureFreeSpace(%s, %d) called with curBytes=%d, maxBytes=%d".format(
-      datasetId, space, currentBytes, maxBytes))
-    val iter = map.entrySet.iterator   // Will give entries in LRU order
-    while (maxBytes - currentBytes < space && iter.hasNext) {
-      val mapEntry = iter.next()
-      val (entryDatasetId, entryPartition) = mapEntry.getKey
-      if (entryDatasetId == datasetId) {
-        // Cannot make space without removing part of the same dataset, or a more recently used one
-        return false
-      }
-      reportEntryDropped(entryDatasetId, entryPartition, mapEntry.getValue)
-      currentBytes -= mapEntry.getValue.size
-      iter.remove()
-    }
-    return true
-  }
-
-  protected def reportEntryDropped(datasetId: Any, partition: Int, entry: Entry) {
-    logInfo("Dropping key (%s, %d) of size %d to make space".format(datasetId, partition, entry.size))
-    // TODO: remove BoundedMemoryCache
-    
-    val (keySpaceId, innerDatasetId) = datasetId.asInstanceOf[(Any, Any)] 
-    innerDatasetId match {
-      case rddId: Int =>
-        SparkEnv.get.cacheTracker.dropEntry(rddId, partition)
-      case broadcastUUID: java.util.UUID =>
-        // TODO: Maybe something should be done if the broadcasted variable falls out of cache  
-      case _ => 
-    }    
-  }
-}
-
-// An entry in our map; stores a cached object and its size in bytes
-private[spark] case class Entry(value: Any, size: Long)
-
-private[spark] object BoundedMemoryCache {
-  /**
-   * Get maximum cache capacity from system configuration
-   */
-   def getMaxBytes: Long = {
-    val memoryFractionToUse = System.getProperty("spark.boundedMemoryCache.memoryFraction", "0.66").toDouble
-    (Runtime.getRuntime.maxMemory * memoryFractionToUse).toLong
-  }
-}
-
diff --git a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala
index a34aee69c162045f10ddc4292ca3a36b87baf88d..6b4a11d6d3f9efc994ab0ccced9b529348542677 100644
--- a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala
+++ b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala
@@ -42,7 +42,13 @@ class SequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable : Cla
       if (classOf[Writable].isAssignableFrom(classManifest[T].erasure)) { 
         classManifest[T].erasure
       } else {
-        implicitly[T => Writable].getClass.getMethods()(0).getReturnType
+        // We get the type of the Writable class by looking at the apply method which converts
+        // from T to Writable. Since we have two apply methods we filter out the one which
+        // is of the form "java.lang.Object apply(java.lang.Object)"
+        implicitly[T => Writable].getClass.getDeclaredMethods().filter(
+            m => m.getReturnType().toString != "java.lang.Object" &&
+                 m.getName() == "apply")(0).getReturnType
+
       }
        // TODO: use something like WritableConverter to avoid reflection
     }
diff --git a/core/src/main/scala/spark/SizeEstimator.scala b/core/src/main/scala/spark/SizeEstimator.scala
index 7c3e8640e96d5b69d7779a6987ad7a661abfcaa5..d4e115725031dd191cb1b3a57b3030550df53840 100644
--- a/core/src/main/scala/spark/SizeEstimator.scala
+++ b/core/src/main/scala/spark/SizeEstimator.scala
@@ -9,7 +9,6 @@ import java.util.Random
 
 import javax.management.MBeanServer
 import java.lang.management.ManagementFactory
-import com.sun.management.HotSpotDiagnosticMXBean
 
 import scala.collection.mutable.ArrayBuffer
 
@@ -76,12 +75,20 @@ private[spark] object SizeEstimator extends Logging {
     if (System.getProperty("spark.test.useCompressedOops") != null) {
       return System.getProperty("spark.test.useCompressedOops").toBoolean 
     }
+
     try {
       val hotSpotMBeanName = "com.sun.management:type=HotSpotDiagnostic"
       val server = ManagementFactory.getPlatformMBeanServer()
+
+      // NOTE: This should throw an exception in non-Sun JVMs
+      val hotSpotMBeanClass = Class.forName("com.sun.management.HotSpotDiagnosticMXBean")
+      val getVMMethod = hotSpotMBeanClass.getDeclaredMethod("getVMOption",
+          Class.forName("java.lang.String"))
+
       val bean = ManagementFactory.newPlatformMXBeanProxy(server, 
-        hotSpotMBeanName, classOf[HotSpotDiagnosticMXBean])
-      return bean.getVMOption("UseCompressedOops").getValue.toBoolean
+        hotSpotMBeanName, hotSpotMBeanClass)
+      // TODO: We could use reflection on the VMOption returned ?
+      return getVMMethod.invoke(bean, "UseCompressedOops").toString.contains("true")
     } catch {
       case e: Exception => {
         // Guess whether they've enabled UseCompressedOops based on whether maxMemory < 32 GB
diff --git a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala
index 340920025b35b88caf3e7a93c3795cb1b83fb463..37524a7c82f8b347d7fbe5f571a9647ff366bb68 100644
--- a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala
+++ b/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala
@@ -104,9 +104,25 @@ private[spark] class WorkerArguments(args: Array[String]) {
   }
 
   def inferDefaultMemory(): Int = {
-    val bean = ManagementFactory.getOperatingSystemMXBean
-                                .asInstanceOf[com.sun.management.OperatingSystemMXBean]
-    val totalMb = (bean.getTotalPhysicalMemorySize / 1024 / 1024).toInt
+    val ibmVendor = System.getProperty("java.vendor").contains("IBM")
+    var totalMb = 0
+    try {
+      val bean = ManagementFactory.getOperatingSystemMXBean()
+      if (ibmVendor) {
+        val beanClass = Class.forName("com.ibm.lang.management.OperatingSystemMXBean")
+        val method = beanClass.getDeclaredMethod("getTotalPhysicalMemory")
+        totalMb = (method.invoke(bean).asInstanceOf[Long] / 1024 / 1024).toInt
+      } else {
+        val beanClass = Class.forName("com.sun.management.OperatingSystemMXBean")
+        val method = beanClass.getDeclaredMethod("getTotalPhysicalMemorySize")
+        totalMb = (method.invoke(bean).asInstanceOf[Long] / 1024 / 1024).toInt
+      }
+    } catch {
+      case e: Exception => {
+        totalMb = 2*1024
+        System.out.println("Failed to get total physical memory. Using " + totalMb + " MB")
+      }
+    }
     // Leave out 1 GB for the operating system, but don't return a negative memory size
     math.max(totalMb - 1024, 512)
   }
diff --git a/core/src/test/scala/spark/BoundedMemoryCacheSuite.scala b/core/src/test/scala/spark/BoundedMemoryCacheSuite.scala
deleted file mode 100644
index 37cafd1e8e3abb473d7edd352c40185533071a04..0000000000000000000000000000000000000000
--- a/core/src/test/scala/spark/BoundedMemoryCacheSuite.scala
+++ /dev/null
@@ -1,58 +0,0 @@
-package spark
-
-import org.scalatest.FunSuite
-import org.scalatest.PrivateMethodTester
-import org.scalatest.matchers.ShouldMatchers
-
-// TODO: Replace this with a test of MemoryStore
-class BoundedMemoryCacheSuite extends FunSuite with PrivateMethodTester with ShouldMatchers {
-  test("constructor test") {
-    val cache = new BoundedMemoryCache(60)
-    expect(60)(cache.getCapacity)
-  }
-
-  test("caching") {
-    // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case 
-    val oldArch = System.setProperty("os.arch", "amd64")
-    val oldOops = System.setProperty("spark.test.useCompressedOops", "true")
-    val initialize = PrivateMethod[Unit]('initialize)
-    SizeEstimator invokePrivate initialize()
-
-    val cache = new BoundedMemoryCache(60) {
-      //TODO sorry about this, but there is not better way how to skip 'cacheTracker.dropEntry'
-      override protected def reportEntryDropped(datasetId: Any, partition: Int, entry: Entry) {
-        logInfo("Dropping key (%s, %d) of size %d to make space".format(datasetId, partition, entry.size))
-      }
-    }
-
-    // NOTE: The String class definition changed in JDK 7 to exclude the int fields count and length
-    // This means that the size of strings will be lesser by 8 bytes in JDK 7 compared to JDK 6.
-    // http://mail.openjdk.java.net/pipermail/core-libs-dev/2012-May/010257.html
-    // Work around to check for either.
-
-    //should be OK
-    cache.put("1", 0, "Meh") should (equal (CachePutSuccess(56)) or equal (CachePutSuccess(48)))
-
-    //we cannot add this to cache (there is not enough space in cache) & we cannot evict the only value from
-    //cache because it's from the same dataset
-    expect(CachePutFailure())(cache.put("1", 1, "Meh"))
-
-    //should be OK, dataset '1' can be evicted from cache
-    cache.put("2", 0, "Meh") should (equal (CachePutSuccess(56)) or equal (CachePutSuccess(48)))
-
-    //should fail, cache should obey it's capacity
-    expect(CachePutFailure())(cache.put("3", 0, "Very_long_and_useless_string"))
-
-    if (oldArch != null) {
-      System.setProperty("os.arch", oldArch)
-    } else {
-      System.clearProperty("os.arch")
-    }
-
-    if (oldOops != null) {
-      System.setProperty("spark.test.useCompressedOops", oldOops)
-    } else {
-      System.clearProperty("spark.test.useCompressedOops")
-    }
-  }
-}
diff --git a/core/src/test/scala/spark/SizeEstimatorSuite.scala b/core/src/test/scala/spark/SizeEstimatorSuite.scala
index 17f366212b7829ff04946fbe6ab6bff10212ff00..e235ef2f67a127b77beb5af3166892a91c81dec4 100644
--- a/core/src/test/scala/spark/SizeEstimatorSuite.scala
+++ b/core/src/test/scala/spark/SizeEstimatorSuite.scala
@@ -3,7 +3,6 @@ package spark
 import org.scalatest.FunSuite
 import org.scalatest.BeforeAndAfterAll
 import org.scalatest.PrivateMethodTester
-import org.scalatest.matchers.ShouldMatchers
 
 class DummyClass1 {}
 
@@ -20,8 +19,17 @@ class DummyClass4(val d: DummyClass3) {
   val x: Int = 0
 }
 
+object DummyString {
+  def apply(str: String) : DummyString = new DummyString(str.toArray)
+}
+class DummyString(val arr: Array[Char]) {
+  override val hashCode: Int = 0
+  // JDK-7 has an extra hash32 field http://hg.openjdk.java.net/jdk7u/jdk7u6/jdk/rev/11987e85555f
+  @transient val hash32: Int = 0
+}
+
 class SizeEstimatorSuite
-  extends FunSuite with BeforeAndAfterAll with PrivateMethodTester with ShouldMatchers {
+  extends FunSuite with BeforeAndAfterAll with PrivateMethodTester {
 
   var oldArch: String = _
   var oldOops: String = _
@@ -45,15 +53,13 @@ class SizeEstimatorSuite
     expect(48)(SizeEstimator.estimate(new DummyClass4(new DummyClass3)))
   }
 
-  // NOTE: The String class definition changed in JDK 7 to exclude the int fields count and length.
-  // This means that the size of strings will be lesser by 8 bytes in JDK 7 compared to JDK 6.
-  // http://mail.openjdk.java.net/pipermail/core-libs-dev/2012-May/010257.html
-  // Work around to check for either.
+  // NOTE: The String class definition varies across JDK versions (1.6 vs. 1.7) and vendors
+  // (Sun vs IBM). Use a DummyString class to make tests deterministic.
   test("strings") {
-    SizeEstimator.estimate("") should (equal (48) or equal (40))
-    SizeEstimator.estimate("a") should (equal (56) or equal (48))
-    SizeEstimator.estimate("ab") should (equal (56) or equal (48))
-    SizeEstimator.estimate("abcdefgh") should (equal(64) or equal(56))
+    expect(40)(SizeEstimator.estimate(DummyString("")))
+    expect(48)(SizeEstimator.estimate(DummyString("a")))
+    expect(48)(SizeEstimator.estimate(DummyString("ab")))
+    expect(56)(SizeEstimator.estimate(DummyString("abcdefgh")))
   }
 
   test("primitive arrays") {
@@ -105,18 +111,16 @@ class SizeEstimatorSuite
     val initialize = PrivateMethod[Unit]('initialize)
     SizeEstimator invokePrivate initialize()
 
-    expect(40)(SizeEstimator.estimate(""))
-    expect(48)(SizeEstimator.estimate("a"))
-    expect(48)(SizeEstimator.estimate("ab"))
-    expect(56)(SizeEstimator.estimate("abcdefgh"))
+    expect(40)(SizeEstimator.estimate(DummyString("")))
+    expect(48)(SizeEstimator.estimate(DummyString("a")))
+    expect(48)(SizeEstimator.estimate(DummyString("ab")))
+    expect(56)(SizeEstimator.estimate(DummyString("abcdefgh")))
 
     resetOrClear("os.arch", arch)
   }
 
-  // NOTE: The String class definition changed in JDK 7 to exclude the int fields count and length.
-  // This means that the size of strings will be lesser by 8 bytes in JDK 7 compared to JDK 6.
-  // http://mail.openjdk.java.net/pipermail/core-libs-dev/2012-May/010257.html
-  // Work around to check for either.
+  // NOTE: The String class definition varies across JDK versions (1.6 vs. 1.7) and vendors
+  // (Sun vs IBM). Use a DummyString class to make tests deterministic.
   test("64-bit arch with no compressed oops") {
     val arch = System.setProperty("os.arch", "amd64")
     val oops = System.setProperty("spark.test.useCompressedOops", "false")
@@ -124,10 +128,10 @@ class SizeEstimatorSuite
     val initialize = PrivateMethod[Unit]('initialize)
     SizeEstimator invokePrivate initialize()
 
-    SizeEstimator.estimate("") should (equal (64) or equal (56))
-    SizeEstimator.estimate("a") should (equal (72) or equal (64))
-    SizeEstimator.estimate("ab") should (equal (72) or equal (64))
-    SizeEstimator.estimate("abcdefgh") should (equal (80) or equal (72))
+    expect(56)(SizeEstimator.estimate(DummyString("")))
+    expect(64)(SizeEstimator.estimate(DummyString("a")))
+    expect(64)(SizeEstimator.estimate(DummyString("ab")))
+    expect(72)(SizeEstimator.estimate(DummyString("abcdefgh")))
 
     resetOrClear("os.arch", arch)
     resetOrClear("spark.test.useCompressedOops", oops)
diff --git a/pom.xml b/pom.xml
index b33cee26b8dbddac8294446f459e240fcad260d5..fe5b1d0ee4b3568713af2651a04b184b44d1ee9e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -489,7 +489,7 @@
           <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-core</artifactId>
-            <version>0.20.205.0</version>
+            <version>1.0.3</version>
           </dependency>
         </dependencies>
       </dependencyManagement>
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 842d0fa96bd034e14e5fc5fed88c124e97eda0b8..7c7c33131a3a73b7107e50955e0e2ffe97df6657 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -10,7 +10,7 @@ import twirl.sbt.TwirlPlugin._
 object SparkBuild extends Build {
   // Hadoop version to build against. For example, "0.20.2", "0.20.205.0", or
   // "1.0.3" for Apache releases, or "0.20.2-cdh3u5" for Cloudera Hadoop.
-  val HADOOP_VERSION = "0.20.205.0"
+  val HADOOP_VERSION = "1.0.3"
   val HADOOP_MAJOR_VERSION = "1"
 
   // For Hadoop 2 versions such as "2.0.0-mr1-cdh4.1.1", set the HADOOP_MAJOR_VERSION to "2"