Merged r289:290 from the hyracks_io_management branch

git-svn-id: https://hyracks.googlecode.com/svn/trunk@291 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCApplicationContext.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCApplicationContext.java
index 9e63446..e20fd90 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCApplicationContext.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCApplicationContext.java
@@ -16,6 +16,14 @@
 
 import java.io.Serializable;
 
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+
 public interface INCApplicationContext extends IApplicationContext {
+    public IHyracksRootContext getRootContext();
+
     public void setDistributedState(Serializable state);
+
+    public void setApplicationObject(Object object);
+
+    public Object getApplicationObject();
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FrameHelper.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FrameHelper.java
index f08ea2a..c8478d6 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FrameHelper.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FrameHelper.java
@@ -14,10 +14,8 @@
  */
 package edu.uci.ics.hyracks.api.comm;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
-
 public class FrameHelper {
-    public static int getTupleCountOffset(IHyracksContext ctx) {
-        return ctx.getFrameSize() - 4;
+    public static int getTupleCountOffset(int frameSize) {
+        return frameSize - 4;
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/IResourceManager.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksCommonContext.java
similarity index 77%
rename from hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/IResourceManager.java
rename to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksCommonContext.java
index 495db70..62112f5 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/IResourceManager.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksCommonContext.java
@@ -12,16 +12,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.api.resources;
+package edu.uci.ics.hyracks.api.context;
 
-import java.io.File;
-import java.io.IOException;
 import java.nio.ByteBuffer;
 
-public interface IResourceManager {
+import edu.uci.ics.hyracks.api.io.IIOManager;
 
+public interface IHyracksCommonContext {
     public ByteBuffer allocateFrame();
 
-    public File createFile(String prefix, String suffix) throws IOException;
+    public int getFrameSize();
 
+    public IIOManager getIOManager();
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksContext.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksJobletContext.java
similarity index 68%
copy from hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksContext.java
copy to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksJobletContext.java
index d90ac60..534507e 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksContext.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksJobletContext.java
@@ -16,15 +16,17 @@
 
 import java.util.UUID;
 
+import edu.uci.ics.hyracks.api.application.INCApplicationContext;
+import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
 import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
-import edu.uci.ics.hyracks.api.resources.IResourceManager;
+import edu.uci.ics.hyracks.api.resources.IDeallocatableRegistry;
 
-public interface IHyracksContext {
-    public IResourceManager getResourceManager();
-
-    public int getFrameSize();
-
-    public ICounterContext getCounterContext();
+public interface IHyracksJobletContext extends IHyracksCommonContext, IWorkspaceFileFactory, IDeallocatableRegistry {
+    public INCApplicationContext getApplicationContext();
 
     public UUID getJobId();
+
+    public int getAttempt();
+
+    public ICounterContext getCounterContext();
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameHelper.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksRootContext.java
similarity index 71%
rename from hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameHelper.java
rename to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksRootContext.java
index 58ff5e1..2ad2e7f 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameHelper.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksRootContext.java
@@ -12,12 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.dataflow.common.comm.io;
+package edu.uci.ics.hyracks.api.context;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
-
-public class FrameHelper {
-    public static int getTupleCountOffset(IHyracksContext ctx) {
-        return ctx.getFrameSize() - 4;
-    }
+public interface IHyracksRootContext extends IHyracksCommonContext {
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksContext.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksStageletContext.java
similarity index 71%
rename from hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksContext.java
rename to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksStageletContext.java
index d90ac60..ae171fe 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksContext.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksStageletContext.java
@@ -16,15 +16,14 @@
 
 import java.util.UUID;
 
+import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
 import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
-import edu.uci.ics.hyracks.api.resources.IResourceManager;
+import edu.uci.ics.hyracks.api.resources.IDeallocatableRegistry;
 
-public interface IHyracksContext {
-    public IResourceManager getResourceManager();
+public interface IHyracksStageletContext extends IHyracksCommonContext, IWorkspaceFileFactory, IDeallocatableRegistry {
+    public IHyracksJobletContext getJobletContext();
 
-    public int getFrameSize();
+    public UUID getStageId();
 
     public ICounterContext getCounterContext();
-
-    public UUID getJobId();
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/INodeController.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/INodeController.java
index 94e5bd3..1842959 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/INodeController.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/INodeController.java
@@ -31,8 +31,8 @@
 
     public NodeCapability getNodeCapability() throws Exception;
 
-    public Map<PortInstanceId, Endpoint> initializeJobletPhase1(String appName, UUID jobId, byte[] plan, UUID stageId,
-            int attempt, Map<ActivityNodeId, Set<Integer>> tasks, Map<OperatorDescriptorId, Set<Integer>> opPartitions)
+    public Map<PortInstanceId, Endpoint> initializeJobletPhase1(String appName, UUID jobId, int attempt, byte[] plan,
+            UUID stageId, Map<ActivityNodeId, Set<Integer>> tasks, Map<OperatorDescriptorId, Set<Integer>> opPartitions)
             throws Exception;
 
     public void initializeJobletPhase2(String appName, UUID jobId, byte[] plan, UUID stageId,
@@ -49,7 +49,8 @@
 
     public void notifyRegistration(IClusterController ccs) throws Exception;
 
-    public void createApplication(String appName, boolean deployHar, byte[] serializedDistributedState) throws Exception;
+    public void createApplication(String appName, boolean deployHar, byte[] serializedDistributedState)
+            throws Exception;
 
     public void destroyApplication(String appName) throws Exception;
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NCConfig.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NCConfig.java
index 64f124b..4a0f64d 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NCConfig.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NCConfig.java
@@ -36,6 +36,9 @@
     @Option(name = "-frame-size", usage = "Frame Size to use for data communication (default: 32768)")
     public int frameSize = 32768;
 
+    @Option(name = "-iodevices", usage = "Comma separated list of IO Device mount points (default: One device in default temp folder)", required = false)
+    public String ioDevices = System.getProperty("java.io.tmpdir");
+
     @Option(name = "-dcache-client-servers", usage = "Sets the list of DCache servers in the format host1:port1,host2:port2,... (default localhost:54583)")
     public String dcacheClientServers = "localhost:54583";
 
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityNode.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityNode.java
index c12e663..39d2eda 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityNode.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityNode.java
@@ -16,7 +16,7 @@
 
 import java.io.Serializable;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
@@ -26,6 +26,6 @@
 
     public IOperatorDescriptor getOwner();
 
-    public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException;
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java
index 25be8b3..63bed19 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java
@@ -22,7 +22,7 @@
 import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
@@ -57,7 +57,7 @@
      * @return data writer.
      * @throws Exception
      */
-    public IFrameWriter createSendSideWriter(IHyracksContext ctx, RecordDescriptor recordDesc,
+    public IFrameWriter createSendSideWriter(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
             IEndpointDataWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
             throws HyracksDataException;
 
@@ -79,7 +79,7 @@
      * @return data reader
      * @throws HyracksDataException
      */
-    public IFrameReader createReceiveSideReader(IHyracksContext ctx, RecordDescriptor recordDesc,
+    public IFrameReader createReceiveSideReader(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
             IConnectionDemultiplexer demux, int index, int nProducerPartitions, int nConsumerPartitions)
             throws HyracksDataException;
 
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/FileHandle.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/FileHandle.java
new file mode 100644
index 0000000..e0e7681
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/FileHandle.java
@@ -0,0 +1,81 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.api.io;
+
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+
+public class FileHandle {
+    private final FileReference fileRef;
+
+    private RandomAccessFile raf;
+
+    private FileChannel channel;
+
+    public FileHandle(FileReference fileRef) {
+        this.fileRef = fileRef;
+    }
+
+    public void open(IIOManager.FileReadWriteMode rwMode, IIOManager.FileSyncMode syncMode) throws IOException {
+        String mode;
+        switch (rwMode) {
+            case READ_ONLY:
+                mode = "r";
+                break;
+
+            case READ_WRITE:
+                switch (syncMode) {
+                    case METADATA_ASYNC_DATA_ASYNC:
+                        mode = "rw";
+                        break;
+
+                    case METADATA_ASYNC_DATA_SYNC:
+                        mode = "rwd";
+                        break;
+
+                    case METADATA_SYNC_DATA_SYNC:
+                        mode = "rws";
+                        break;
+
+                    default:
+                        throw new IllegalArgumentException();
+                }
+                break;
+
+            default:
+                throw new IllegalArgumentException();
+        }
+        raf = new RandomAccessFile(fileRef.getFile(), mode);
+        channel = raf.getChannel();
+    }
+
+    public void close() throws IOException {
+        channel.close();
+        raf.close();
+    }
+
+    public FileReference getFileReference() {
+        return fileRef;
+    }
+
+    public RandomAccessFile getRandomAccessFile() {
+        return raf;
+    }
+
+    public FileChannel getFileChannel() {
+        return channel;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/FileReference.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/FileReference.java
new file mode 100644
index 0000000..e9ba2c3
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/FileReference.java
@@ -0,0 +1,54 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.api.io;
+
+import java.io.File;
+import java.io.Serializable;
+
+public final class FileReference implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final File file;
+
+    public FileReference(IODeviceHandle dev, String devRelPath) {
+        file = new File(dev.getPath(), devRelPath);
+    }
+
+    public FileReference(File file) {
+        this.file = file;
+    }
+
+    public File getFile() {
+        return file;
+    }
+
+    @Override
+    public String toString() {
+        return file.getAbsolutePath();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof FileReference)) {
+            return false;
+        }
+        return file.equals(((FileReference) o).file);
+    }
+
+    @Override
+    public int hashCode() {
+        return file.hashCode();
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameHelper.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/IIOFuture.java
similarity index 71%
copy from hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameHelper.java
copy to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/IIOFuture.java
index 58ff5e1..45b919f 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameHelper.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/IIOFuture.java
@@ -12,12 +12,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.dataflow.common.comm.io;
+package edu.uci.ics.hyracks.api.io;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
-public class FrameHelper {
-    public static int getTupleCountOffset(IHyracksContext ctx) {
-        return ctx.getFrameSize() - 4;
-    }
+public interface IIOFuture {
+    public int synchronize() throws HyracksDataException, InterruptedException;
+
+    public boolean isComplete();
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/IIOManager.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/IIOManager.java
new file mode 100644
index 0000000..939cfc2
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/IIOManager.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.api.io;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IIOManager {
+    public enum FileReadWriteMode {
+        READ_ONLY,
+        READ_WRITE
+    }
+
+    public enum FileSyncMode {
+        METADATA_SYNC_DATA_SYNC,
+        METADATA_ASYNC_DATA_SYNC,
+        METADATA_ASYNC_DATA_ASYNC
+    }
+
+    public List<IODeviceHandle> getIODevices();
+
+    public FileHandle open(FileReference fileRef, FileReadWriteMode rwMode, FileSyncMode syncMode)
+            throws HyracksDataException;
+
+    public int syncWrite(FileHandle fHandle, long offset, ByteBuffer data) throws HyracksDataException;
+
+    public int syncRead(FileHandle fHandle, long offset, ByteBuffer data) throws HyracksDataException;
+
+    public IIOFuture asyncWrite(FileHandle fHandle, long offset, ByteBuffer data);
+
+    public IIOFuture asyncRead(FileHandle fHandle, long offset, ByteBuffer data);
+
+    public void close(FileHandle fHandle) throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/IODeviceHandle.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/IODeviceHandle.java
new file mode 100644
index 0000000..16ff19c
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/IODeviceHandle.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.api.io;
+
+import java.io.File;
+import java.io.Serializable;
+
+public class IODeviceHandle implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final File path;
+
+    private final String workAreaPath;
+
+    public IODeviceHandle(File path, String workAreaPath) {
+        this.path = path;
+        workAreaPath = workAreaPath.trim();
+        if (workAreaPath.endsWith(File.separator)) {
+            workAreaPath = workAreaPath.substring(0, workAreaPath.length() - 1);
+        }
+        this.workAreaPath = workAreaPath;
+    }
+
+    public File getPath() {
+        return path;
+    }
+
+    public String getWorkAreaPath() {
+        return workAreaPath;
+    }
+
+    public FileReference createFileReference(String relPath) {
+        return new FileReference(this, relPath);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameHelper.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/IWorkspaceFileFactory.java
similarity index 71%
copy from hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameHelper.java
copy to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/IWorkspaceFileFactory.java
index 58ff5e1..1311ec5 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameHelper.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/IWorkspaceFileFactory.java
@@ -12,12 +12,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.dataflow.common.comm.io;
+package edu.uci.ics.hyracks.api.io;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
-public class FrameHelper {
-    public static int getTupleCountOffset(IHyracksContext ctx) {
-        return ctx.getFrameSize() - 4;
-    }
+public interface IWorkspaceFileFactory {
+    public FileReference createWorkspaceFile(String prefix) throws HyracksDataException;
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/IResourceManager.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/IDeallocatable.java
similarity index 74%
copy from hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/IResourceManager.java
copy to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/IDeallocatable.java
index 495db70..e5b3ed8 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/IResourceManager.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/IDeallocatable.java
@@ -14,14 +14,6 @@
  */
 package edu.uci.ics.hyracks.api.resources;
 
-import java.io.File;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public interface IResourceManager {
-
-    public ByteBuffer allocateFrame();
-
-    public File createFile(String prefix, String suffix) throws IOException;
-
+public interface IDeallocatable {
+    public void deallocate();
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/IResourceManager.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/IDeallocatableRegistry.java
similarity index 74%
copy from hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/IResourceManager.java
copy to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/IDeallocatableRegistry.java
index 495db70..e130607 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/IResourceManager.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/IDeallocatableRegistry.java
@@ -14,14 +14,6 @@
  */
 package edu.uci.ics.hyracks.api.resources;
 
-import java.io.File;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public interface IResourceManager {
-
-    public ByteBuffer allocateFrame();
-
-    public File createFile(String prefix, String suffix) throws IOException;
-
+public interface IDeallocatableRegistry {
+    public void registerDeallocatable(IDeallocatable deallocatable);
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java b/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
index a9fd347..f577c97 100644
--- a/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
+++ b/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
@@ -547,7 +547,7 @@
 
         @Override
         public Map<PortInstanceId, Endpoint> execute(INodeController node) throws Exception {
-            return node.initializeJobletPhase1(appName, jobId, serialize(plan), stageId, attempt, tasks, opPartitions);
+            return node.initializeJobletPhase1(appName, jobId, attempt, serialize(plan), stageId, tasks, opPartitions);
         }
 
         @Override
diff --git a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
index b48be8f..ccbc558 100644
--- a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
+++ b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
@@ -14,6 +14,7 @@
  */
 package edu.uci.ics.hyracks.control.nc;
 
+import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
@@ -21,28 +22,56 @@
 import java.util.UUID;
 import java.util.concurrent.Executor;
 
+import edu.uci.ics.hyracks.api.application.INCApplicationContext;
+import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
 import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
+import edu.uci.ics.hyracks.api.resources.IDeallocatable;
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
+import edu.uci.ics.hyracks.control.nc.io.ManagedWorkspaceFileFactory;
+import edu.uci.ics.hyracks.control.nc.job.profiling.CounterContext;
+import edu.uci.ics.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
 
-public class Joblet {
+public class Joblet implements IHyracksJobletContext {
     private static final long serialVersionUID = 1L;
 
     private final NodeControllerService nodeController;
 
+    private final INCApplicationContext appCtx;
+
     private final UUID jobId;
 
+    private final int attempt;
+
     private final Map<UUID, Stagelet> stageletMap;
 
     private final Map<OperatorDescriptorId, Map<Integer, IOperatorEnvironment>> envMap;
 
-    public Joblet(NodeControllerService nodeController, UUID jobId) {
+    private final ICounterContext counterContext;
+
+    private final DefaultDeallocatableRegistry deallocatableRegistry;
+
+    private final IWorkspaceFileFactory fileFactory;
+
+    public Joblet(NodeControllerService nodeController, UUID jobId, int attempt, INCApplicationContext appCtx) {
         this.nodeController = nodeController;
+        this.appCtx = appCtx;
         this.jobId = jobId;
+        this.attempt = attempt;
         stageletMap = new HashMap<UUID, Stagelet>();
         envMap = new HashMap<OperatorDescriptorId, Map<Integer, IOperatorEnvironment>>();
+        counterContext = new CounterContext(getJobId() + "." + nodeController.getId());
+        deallocatableRegistry = new DefaultDeallocatableRegistry();
+        fileFactory = new ManagedWorkspaceFileFactory(this, (IOManager) appCtx.getRootContext().getIOManager());
     }
 
+    @Override
     public UUID getJobId() {
         return jobId;
     }
@@ -118,4 +147,48 @@
             }
         }
     }
+
+    @Override
+    public INCApplicationContext getApplicationContext() {
+        return null;
+    }
+
+    @Override
+    public int getAttempt() {
+        return attempt;
+    }
+
+    @Override
+    public ICounterContext getCounterContext() {
+        return counterContext;
+    }
+
+    @Override
+    public void registerDeallocatable(IDeallocatable deallocatable) {
+        deallocatableRegistry.registerDeallocatable(deallocatable);
+    }
+
+    public void close() {
+        deallocatableRegistry.close();
+    }
+
+    @Override
+    public ByteBuffer allocateFrame() {
+        return appCtx.getRootContext().allocateFrame();
+    }
+
+    @Override
+    public int getFrameSize() {
+        return appCtx.getRootContext().getFrameSize();
+    }
+
+    @Override
+    public IIOManager getIOManager() {
+        return appCtx.getRootContext().getIOManager();
+    }
+
+    @Override
+    public FileReference createWorkspaceFile(String prefix) throws HyracksDataException {
+        return fileFactory.createWorkspaceFile(prefix);
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
index 9733d94..19c8ed8 100644
--- a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
@@ -30,6 +30,7 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.StringTokenizer;
 import java.util.Timer;
 import java.util.TimerTask;
 import java.util.UUID;
@@ -46,11 +47,13 @@
 import org.apache.http.client.methods.HttpGet;
 import org.apache.http.impl.client.DefaultHttpClient;
 
+import edu.uci.ics.hyracks.api.application.INCApplicationContext;
 import edu.uci.ics.hyracks.api.comm.Endpoint;
 import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.control.IClusterController;
 import edu.uci.ics.hyracks.api.control.INodeController;
 import edu.uci.ics.hyracks.api.control.NCConfig;
@@ -70,6 +73,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.io.IODeviceHandle;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobPlan;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
@@ -80,18 +84,20 @@
 import edu.uci.ics.hyracks.control.nc.application.NCApplicationContext;
 import edu.uci.ics.hyracks.control.nc.comm.ConnectionManager;
 import edu.uci.ics.hyracks.control.nc.comm.DemuxDataReceiveListenerFactory;
-import edu.uci.ics.hyracks.control.nc.runtime.DelegateHyracksContext;
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
 import edu.uci.ics.hyracks.control.nc.runtime.OperatorRunnable;
 import edu.uci.ics.hyracks.control.nc.runtime.RootHyracksContext;
 
 public class NodeControllerService extends AbstractRemoteService implements INodeController {
+    private static Logger LOGGER = Logger.getLogger(NodeControllerService.class.getName());
+
     private static final long serialVersionUID = 1L;
 
     private NCConfig ncConfig;
 
     private final String id;
 
-    private final IHyracksContext ctx;
+    private final IHyracksRootContext ctx;
 
     private final NodeCapability nodeCapability;
 
@@ -101,9 +107,9 @@
 
     private IClusterController ccs;
 
-    private Map<UUID, Joblet> jobletMap;
+    private final Map<UUID, Joblet> jobletMap;
 
-    private Executor executor;
+    private final Executor executor;
 
     private NodeParameters nodeParameters;
 
@@ -114,21 +120,29 @@
     public NodeControllerService(NCConfig ncConfig) throws Exception {
         this.ncConfig = ncConfig;
         id = ncConfig.nodeId;
-        this.ctx = new RootHyracksContext(ncConfig.frameSize);
+        executor = Executors.newCachedThreadPool();
+        this.ctx = new RootHyracksContext(ncConfig.frameSize, new IOManager(getDevices(ncConfig.ioDevices), executor));
         if (id == null) {
             throw new Exception("id not set");
         }
         nodeCapability = computeNodeCapability();
         connectionManager = new ConnectionManager(ctx, getIpAddress(ncConfig));
         jobletMap = new HashMap<UUID, Joblet>();
-        executor = Executors.newCachedThreadPool();
         timer = new Timer(true);
         serverCtx = new ServerContext(ServerContext.ServerType.NODE_CONTROLLER, new File(new File(
                 NodeControllerService.class.getName()), id));
         applications = new Hashtable<String, NCApplicationContext>();
     }
 
-    private static Logger LOGGER = Logger.getLogger(NodeControllerService.class.getName());
+    private static List<IODeviceHandle> getDevices(String ioDevices) {
+        List<IODeviceHandle> devices = new ArrayList<IODeviceHandle>();
+        StringTokenizer tok = new StringTokenizer(ioDevices, ",");
+        while (tok.hasMoreElements()) {
+            String devPath = tok.nextToken().trim();
+            devices.add(new IODeviceHandle(new File(devPath), "."));
+        }
+        return devices;
+    }
 
     @Override
     public void start() throws Exception {
@@ -194,13 +208,13 @@
     }
 
     @Override
-    public Map<PortInstanceId, Endpoint> initializeJobletPhase1(String appName, UUID jobId, byte[] planBytes,
-            UUID stageId, int attempt, Map<ActivityNodeId, Set<Integer>> tasks,
+    public Map<PortInstanceId, Endpoint> initializeJobletPhase1(String appName, UUID jobId, int attempt,
+            byte[] planBytes, UUID stageId, Map<ActivityNodeId, Set<Integer>> tasks,
             Map<OperatorDescriptorId, Set<Integer>> opPartitions) throws Exception {
         try {
             LOGGER.log(Level.INFO, String.valueOf(jobId) + "[" + id + ":" + stageId + "]: Initializing Joblet Phase 1");
 
-            ApplicationContext appCtx = applications.get(appName);
+            NCApplicationContext appCtx = applications.get(appName);
             final JobPlan plan = (JobPlan) appCtx.deserialize(planBytes);
 
             IRecordDescriptorProvider rdp = new IRecordDescriptorProvider() {
@@ -215,14 +229,11 @@
                 }
             };
 
-            final Joblet joblet = getLocalJoblet(jobId);
+            final Joblet joblet = getOrCreateLocalJoblet(jobId, attempt, appCtx);
 
             Stagelet stagelet = new Stagelet(joblet, stageId, attempt, id);
             joblet.setStagelet(stageId, stagelet);
 
-            IHyracksContext stageletContext = new DelegateHyracksContext(ctx, jobId,
-                    stagelet.getStageletCounterContext());
-
             final Map<PortInstanceId, Endpoint> portMap = new HashMap<PortInstanceId, Endpoint>();
             Map<OperatorInstanceId, OperatorRunnable> honMap = stagelet.getOperatorMap();
 
@@ -236,9 +247,9 @@
                 IOperatorDescriptor op = han.getOwner();
                 List<IConnectorDescriptor> inputs = plan.getTaskInputs(hanId);
                 for (int i : tasks.get(hanId)) {
-                    IOperatorNodePushable hon = han.createPushRuntime(stageletContext, joblet.getEnvironment(op, i),
-                            rdp, i, opPartitions.get(op.getOperatorId()).size());
-                    OperatorRunnable or = new OperatorRunnable(stageletContext, hon);
+                    IOperatorNodePushable hon = han.createPushRuntime(stagelet, joblet.getEnvironment(op, i), rdp, i,
+                            opPartitions.get(op.getOperatorId()).size());
+                    OperatorRunnable or = new OperatorRunnable(stagelet, hon);
                     stagelet.setOperator(op.getOperatorId(), i, or);
                     if (inputs != null) {
                         for (int j = 0; j < inputs.size(); ++j) {
@@ -252,8 +263,8 @@
                                     .getOperatorId();
                             Endpoint endpoint = new Endpoint(connectionManager.getNetworkAddress(), i);
                             endpointList.add(endpoint);
-                            DemuxDataReceiveListenerFactory drlf = new DemuxDataReceiveListenerFactory(stageletContext,
-                                    jobId, stageId);
+                            DemuxDataReceiveListenerFactory drlf = new DemuxDataReceiveListenerFactory(stagelet, jobId,
+                                    stageId);
                             connectionManager.acceptConnection(endpoint.getEndpointId(), drlf);
                             PortInstanceId piId = new PortInstanceId(op.getOperatorId(), Direction.INPUT, plan
                                     .getTaskInputMap().get(hanId).get(j), i);
@@ -261,8 +272,8 @@
                                 LOGGER.finest("Created endpoint " + piId + " -> " + endpoint);
                             }
                             portMap.put(piId, endpoint);
-                            IFrameReader reader = createReader(stageletContext, conn, drlf, i, plan, stagelet,
-                                    opPartitions.get(producerOpId).size(), opPartitions.get(consumerOpId).size());
+                            IFrameReader reader = createReader(stagelet, conn, drlf, i, plan, stagelet, opPartitions
+                                    .get(producerOpId).size(), opPartitions.get(consumerOpId).size());
                             or.setFrameReader(reader);
                         }
                     }
@@ -279,7 +290,7 @@
         }
     }
 
-    private IFrameReader createReader(final IHyracksContext stageletContext, final IConnectorDescriptor conn,
+    private IFrameReader createReader(final IHyracksStageletContext stageletContext, final IConnectorDescriptor conn,
             IConnectionDemultiplexer demux, final int receiverIndex, JobPlan plan, final Stagelet stagelet,
             int nProducerCount, int nConsumerCount) throws HyracksDataException {
         final IFrameReader reader = conn.createReceiveSideReader(stageletContext, plan.getJobSpecification()
@@ -326,13 +337,8 @@
             final JobPlan plan = (JobPlan) appCtx.deserialize(planBytes);
 
             final Joblet ji = getLocalJoblet(jobId);
-            Stagelet si = (Stagelet) ji.getStagelet(stageId);
-            final Map<OperatorInstanceId, OperatorRunnable> honMap = si.getOperatorMap();
-
             final Stagelet stagelet = (Stagelet) ji.getStagelet(stageId);
-
-            final IHyracksContext stageletContext = new DelegateHyracksContext(ctx, jobId,
-                    stagelet.getStageletCounterContext());
+            final Map<OperatorInstanceId, OperatorRunnable> honMap = stagelet.getOperatorMap();
 
             final JobSpecification spec = plan.getJobSpecification();
 
@@ -363,12 +369,11 @@
                                     if (LOGGER.isLoggable(Level.FINEST)) {
                                         LOGGER.finest("Probed endpoint " + piId + " -> " + ep);
                                     }
-                                    return createWriter(stageletContext, connectionManager.connect(
-                                            ep.getNetworkAddress(), ep.getEndpointId(), senderIndex), plan, conn,
-                                            senderIndex, index, stagelet);
+                                    return createWriter(stagelet, connectionManager.connect(ep.getNetworkAddress(),
+                                            ep.getEndpointId(), senderIndex), plan, conn, senderIndex, index, stagelet);
                                 }
                             };
-                            or.setFrameWriter(j, conn.createSendSideWriter(stageletContext, plan.getJobSpecification()
+                            or.setFrameWriter(j, conn.createSendSideWriter(stagelet, plan.getJobSpecification()
                                     .getConnectorRecordDescriptor(conn), edwFactory, i, opPartitions.get(producerOpId)
                                     .size(), opPartitions.get(consumerOpId).size()), spec
                                     .getConnectorRecordDescriptor(conn));
@@ -383,9 +388,9 @@
         }
     }
 
-    private IFrameWriter createWriter(final IHyracksContext stageletContext, final IFrameWriter writer, JobPlan plan,
-            final IConnectorDescriptor conn, final int senderIndex, final int receiverIndex, final Stagelet stagelet)
-            throws HyracksDataException {
+    private IFrameWriter createWriter(final IHyracksStageletContext stageletContext, final IFrameWriter writer,
+            JobPlan plan, final IConnectorDescriptor conn, final int senderIndex, final int receiverIndex,
+            final Stagelet stagelet) throws HyracksDataException {
         return plan.getJobFlags().contains(JobFlag.PROFILE_RUNTIME) ? new IFrameWriter() {
             private ICounter openCounter = stageletContext.getCounterContext().getCounter(
                     conn.getConnectorId().getId() + ".sender." + senderIndex + "." + receiverIndex + ".open", true);
@@ -433,8 +438,14 @@
 
     private synchronized Joblet getLocalJoblet(UUID jobId) throws Exception {
         Joblet ji = jobletMap.get(jobId);
+        return ji;
+    }
+
+    private synchronized Joblet getOrCreateLocalJoblet(UUID jobId, int attempt, INCApplicationContext appCtx)
+            throws Exception {
+        Joblet ji = jobletMap.get(jobId);
         if (ji == null) {
-            ji = new Joblet(this, jobId);
+            ji = new Joblet(this, jobId, attempt, appCtx);
             jobletMap.put(jobId, ji);
         }
         return ji;
@@ -449,7 +460,10 @@
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Cleaning up after job: " + jobId);
         }
-        jobletMap.remove(jobId);
+        Joblet joblet = jobletMap.remove(jobId);
+        if (joblet != null) {
+            joblet.close();
+        }
         connectionManager.dumpStats();
     }
 
@@ -551,6 +565,7 @@
             Stagelet stagelet = ji.getStagelet(stageId);
             if (stagelet != null) {
                 stagelet.abort();
+                stagelet.close();
                 connectionManager.abortConnections(jobId, stageId);
             }
         }
@@ -564,7 +579,7 @@
             if (applications.containsKey(appName)) {
                 throw new HyracksException("Duplicate application with name: " + appName + " being created.");
             }
-            appCtx = new NCApplicationContext(serverCtx, appName);
+            appCtx = new NCApplicationContext(serverCtx, ctx, appName);
             applications.put(appName, appCtx);
         }
         if (deployHar) {
diff --git a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Stagelet.java b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Stagelet.java
index 0f53f26..3565445 100644
--- a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Stagelet.java
+++ b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Stagelet.java
@@ -14,6 +14,7 @@
  */
 package edu.uci.ics.hyracks.control.nc;
 
+import java.nio.ByteBuffer;
 import java.rmi.RemoteException;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -26,12 +27,23 @@
 import java.util.logging.Logger;
 
 import edu.uci.ics.hyracks.api.comm.Endpoint;
+import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
 import edu.uci.ics.hyracks.api.dataflow.OperatorInstanceId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
+import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
+import edu.uci.ics.hyracks.api.resources.IDeallocatable;
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
+import edu.uci.ics.hyracks.control.nc.io.ManagedWorkspaceFileFactory;
 import edu.uci.ics.hyracks.control.nc.job.profiling.CounterContext;
+import edu.uci.ics.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
 import edu.uci.ics.hyracks.control.nc.runtime.OperatorRunnable;
 
-public class Stagelet {
+public class Stagelet implements IHyracksStageletContext {
     private static final long serialVersionUID = 1L;
 
     private static final Logger LOGGER = Logger.getLogger(Stagelet.class.getName());
@@ -46,6 +58,8 @@
 
     private final CounterContext stageletCounterContext;
 
+    private final IWorkspaceFileFactory fileFactory;
+
     private List<Endpoint> endpointList;
 
     private boolean started;
@@ -54,6 +68,8 @@
 
     private final Set<OperatorInstanceId> pendingOperators;
 
+    private final DefaultDeallocatableRegistry deallocatableRegistry;
+
     public Stagelet(Joblet joblet, UUID stageId, int attempt, String nodeId) throws RemoteException {
         this.joblet = joblet;
         this.stageId = stageId;
@@ -62,6 +78,8 @@
         started = false;
         honMap = new HashMap<OperatorInstanceId, OperatorRunnable>();
         stageletCounterContext = new CounterContext(joblet.getJobId() + "." + stageId + "." + nodeId);
+        deallocatableRegistry = new DefaultDeallocatableRegistry();
+        fileFactory = new ManagedWorkspaceFileFactory(this, (IOManager) joblet.getIOManager());
     }
 
     public void setOperator(OperatorDescriptorId odId, int partition, OperatorRunnable hon) {
@@ -142,6 +160,7 @@
             try {
                 Map<String, Long> stats = new TreeMap<String, Long>();
                 dumpProfile(stats);
+                close();
                 joblet.notifyStageletComplete(stageId, attempt, stats);
             } catch (Exception e) {
                 e.printStackTrace();
@@ -167,4 +186,48 @@
     public void dumpProfile(Map<String, Long> counterDump) {
         stageletCounterContext.dump(counterDump);
     }
+
+    @Override
+    public IHyracksJobletContext getJobletContext() {
+        return joblet;
+    }
+
+    @Override
+    public UUID getStageId() {
+        return stageId;
+    }
+
+    @Override
+    public ICounterContext getCounterContext() {
+        return stageletCounterContext;
+    }
+
+    @Override
+    public void registerDeallocatable(IDeallocatable deallocatable) {
+        deallocatableRegistry.registerDeallocatable(deallocatable);
+    }
+
+    public void close() {
+        deallocatableRegistry.close();
+    }
+
+    @Override
+    public ByteBuffer allocateFrame() {
+        return joblet.allocateFrame();
+    }
+
+    @Override
+    public int getFrameSize() {
+        return joblet.getFrameSize();
+    }
+
+    @Override
+    public IIOManager getIOManager() {
+        return joblet.getIOManager();
+    }
+
+    @Override
+    public FileReference createWorkspaceFile(String prefix) throws HyracksDataException {
+        return fileFactory.createWorkspaceFile(prefix);
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/application/NCApplicationContext.java b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/application/NCApplicationContext.java
index c894b4a..349743b 100644
--- a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/application/NCApplicationContext.java
+++ b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/application/NCApplicationContext.java
@@ -5,12 +5,18 @@
 
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
 import edu.uci.ics.hyracks.api.application.INCBootstrap;
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
 import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
 import edu.uci.ics.hyracks.control.common.context.ServerContext;
 
 public class NCApplicationContext extends ApplicationContext implements INCApplicationContext {
-    public NCApplicationContext(ServerContext serverCtx, String appName) throws IOException {
+    private final IHyracksRootContext rootCtx;
+    private Object appObject;
+
+    public NCApplicationContext(ServerContext serverCtx, IHyracksRootContext rootCtx, String appName)
+            throws IOException {
         super(serverCtx, appName);
+        this.rootCtx = rootCtx;
     }
 
     @Override
@@ -30,4 +36,19 @@
             bootstrap.stop();
         }
     }
+
+    @Override
+    public IHyracksRootContext getRootContext() {
+        return rootCtx;
+    }
+
+    @Override
+    public void setApplicationObject(Object object) {
+        this.appObject = object;
+    }
+
+    @Override
+    public Object getApplicationObject() {
+        return appObject;
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/ConnectionEntry.java b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/ConnectionEntry.java
index 20d41d6..e49f435 100644
--- a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/ConnectionEntry.java
+++ b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/ConnectionEntry.java
@@ -24,7 +24,7 @@
 
 import edu.uci.ics.hyracks.api.comm.IConnectionEntry;
 import edu.uci.ics.hyracks.api.comm.IDataReceiveListener;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
 
 public class ConnectionEntry implements IConnectionEntry {
     private static final Logger LOGGER = Logger.getLogger(ConnectionEntry.class.getName());
@@ -47,11 +47,11 @@
 
     private boolean aborted;
 
-    public ConnectionEntry(IHyracksContext ctx, SocketChannel socketChannel, SelectionKey key) {
+    public ConnectionEntry(IHyracksRootContext ctx, SocketChannel socketChannel, SelectionKey key) {
         this.socketChannel = socketChannel;
-        readBuffer = ctx.getResourceManager().allocateFrame();
+        readBuffer = ctx.allocateFrame();
         readBuffer.clear();
-        writeBuffer = ctx.getResourceManager().allocateFrame();
+        writeBuffer = ctx.allocateFrame();
         writeBuffer.clear();
         this.key = key;
     }
diff --git a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/ConnectionManager.java b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/ConnectionManager.java
index 927d67b..717144d 100644
--- a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/ConnectionManager.java
+++ b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/ConnectionManager.java
@@ -42,7 +42,7 @@
 import edu.uci.ics.hyracks.api.comm.IDataReceiveListenerFactory;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.comm.NetworkAddress;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 public class ConnectionManager {
@@ -54,7 +54,7 @@
 
     private ServerSocketChannel serverSocketChannel;
 
-    private final IHyracksContext ctx;
+    private final IHyracksRootContext ctx;
 
     private final Map<UUID, IDataReceiveListenerFactory> pendingConnectionReceivers;
 
@@ -70,7 +70,7 @@
 
     private ByteBuffer emptyFrame;
 
-    public ConnectionManager(IHyracksContext ctx, InetAddress address) throws IOException {
+    public ConnectionManager(IHyracksRootContext ctx, InetAddress address) throws IOException {
         this.ctx = ctx;
         serverSocketChannel = ServerSocketChannel.open();
         ServerSocket serverSocket = serverSocketChannel.socket();
@@ -87,8 +87,8 @@
         dataListenerThread = new DataListenerThread();
         connectionListenerThread = new ConnectionListenerThread();
         initialDataReceiveListener = new InitialDataReceiveListener();
-        emptyFrame = ctx.getResourceManager().allocateFrame();
-        emptyFrame.putInt(FrameHelper.getTupleCountOffset(ctx), 0);
+        emptyFrame = ctx.allocateFrame();
+        emptyFrame.putInt(FrameHelper.getTupleCountOffset(ctx.getFrameSize()), 0);
         connections = new HashSet<IConnectionEntry>();
     }
 
diff --git a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/DemuxDataReceiveListenerFactory.java b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/DemuxDataReceiveListenerFactory.java
index 6b4c041..32c8991 100644
--- a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/DemuxDataReceiveListenerFactory.java
+++ b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/DemuxDataReceiveListenerFactory.java
@@ -27,22 +27,22 @@
 import edu.uci.ics.hyracks.api.comm.IConnectionEntry;
 import edu.uci.ics.hyracks.api.comm.IDataReceiveListener;
 import edu.uci.ics.hyracks.api.comm.IDataReceiveListenerFactory;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 public class DemuxDataReceiveListenerFactory implements IDataReceiveListenerFactory, IConnectionDemultiplexer,
         IDataReceiveListener {
     private static final Logger LOGGER = Logger.getLogger(DemuxDataReceiveListenerFactory.class.getName());
 
-    private final IHyracksContext ctx;
     private final BitSet readyBits;
+    private final int frameSize;
     private IConnectionEntry senders[];
     private int openSenderCount;
     private UUID jobId;
     private UUID stageId;
 
-    public DemuxDataReceiveListenerFactory(IHyracksContext ctx, UUID jobId, UUID stageId) {
-        this.ctx = ctx;
+    public DemuxDataReceiveListenerFactory(IHyracksStageletContext ctx, UUID jobId, UUID stageId) {
+        frameSize = ctx.getFrameSize();
         this.jobId = jobId;
         this.stageId = stageId;
         readyBits = new BitSet();
@@ -63,7 +63,7 @@
         ByteBuffer buffer = entry.getReadBuffer();
         buffer.flip();
         int dataLen = buffer.remaining();
-        if (dataLen >= ctx.getFrameSize() || entry.aborted()) {
+        if (dataLen >= frameSize || entry.aborted()) {
             if (LOGGER.isLoggable(Level.FINEST)) {
                 LOGGER.finest("NonDeterministicDataReceiveListener: frame received: sender = " + senderIndex);
             }
diff --git a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/IOFuture.java b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/IOFuture.java
new file mode 100644
index 0000000..bd6462d
--- /dev/null
+++ b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/IOFuture.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.control.nc.io;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public final class IOFuture {
+    private boolean complete;
+
+    private Exception exception;
+
+    public synchronized void reset() {
+        complete = false;
+        exception = null;
+    }
+
+    public synchronized void synchronize() throws HyracksDataException, InterruptedException {
+        while (!complete) {
+            try {
+                wait();
+            } catch (InterruptedException e) {
+                throw e;
+            }
+        }
+        if (exception != null) {
+            throw new HyracksDataException(exception);
+        }
+    }
+
+    public synchronized boolean isComplete() {
+        return complete;
+    }
+
+    public synchronized void notifySuccess() {
+        complete = true;
+        notifyAll();
+    }
+
+    public synchronized void notifyFailure(Exception e) {
+        complete = true;
+        exception = e;
+        notifyAll();
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/IOManager.java b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/IOManager.java
new file mode 100644
index 0000000..57174b5
--- /dev/null
+++ b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/IOManager.java
@@ -0,0 +1,225 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.control.nc.io;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Executor;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.io.FileHandle;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IIOFuture;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.io.IODeviceHandle;
+
+public class IOManager implements IIOManager {
+    private final List<IODeviceHandle> ioDevices;
+
+    private final Executor executor;
+
+    private final List<IODeviceHandle> workAreaIODevices;
+
+    private int workAreaDeviceIndex;
+
+    public IOManager(List<IODeviceHandle> devices, Executor executor) throws HyracksException {
+        this.ioDevices = Collections.unmodifiableList(devices);
+        this.executor = executor;
+        workAreaIODevices = new ArrayList<IODeviceHandle>();
+        for (IODeviceHandle d : ioDevices) {
+            if (d.getWorkAreaPath() != null) {
+                workAreaIODevices.add(d);
+            }
+        }
+        if (workAreaIODevices.isEmpty()) {
+            throw new HyracksException("No devices with work areas found");
+        }
+        workAreaDeviceIndex = 0;
+    }
+
+    @Override
+    public List<IODeviceHandle> getIODevices() {
+        return ioDevices;
+    }
+
+    @Override
+    public FileHandle open(FileReference fileRef, FileReadWriteMode rwMode, FileSyncMode syncMode)
+            throws HyracksDataException {
+        FileHandle fHandle = new FileHandle(fileRef);
+        try {
+            fHandle.open(rwMode, syncMode);
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+        return fHandle;
+    }
+
+    @Override
+    public int syncWrite(FileHandle fHandle, long offset, ByteBuffer data) throws HyracksDataException {
+        try {
+            int n = 0;
+            int remaining = data.remaining();
+            while (remaining > 0) {
+                int len = fHandle.getFileChannel().write(data, offset);
+                if (len < 0) {
+                    throw new HyracksDataException("Error writing to file: " + fHandle.getFileReference().toString());
+                }
+                remaining -= len;
+                n += len;
+            }
+            return n;
+        } catch (HyracksDataException e) {
+            throw e;
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public int syncRead(FileHandle fHandle, long offset, ByteBuffer data) throws HyracksDataException {
+        try {
+            int n = 0;
+            int remaining = data.remaining();
+            while (remaining > 0) {
+                int len = fHandle.getFileChannel().read(data, offset);
+                if (len < 0) {
+                    return -1;
+                }
+                remaining -= len;
+                n += len;
+            }
+            return n;
+        } catch (HyracksDataException e) {
+            throw e;
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public IIOFuture asyncWrite(FileHandle fHandle, long offset, ByteBuffer data) {
+        AsyncWriteRequest req = new AsyncWriteRequest(fHandle, offset, data);
+        executor.execute(req);
+        return req;
+    }
+
+    @Override
+    public IIOFuture asyncRead(FileHandle fHandle, long offset, ByteBuffer data) {
+        AsyncReadRequest req = new AsyncReadRequest(fHandle, offset, data);
+        executor.execute(req);
+        return req;
+    }
+
+    @Override
+    public void close(FileHandle fHandle) throws HyracksDataException {
+        try {
+            fHandle.close();
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    public synchronized FileReference createWorkspaceFile(String prefix) throws HyracksDataException {
+        IODeviceHandle dev = workAreaIODevices.get(workAreaDeviceIndex);
+        workAreaDeviceIndex = (workAreaDeviceIndex + 1) % workAreaIODevices.size();
+        String waPath = dev.getWorkAreaPath();
+        File waf;
+        try {
+            waf = File.createTempFile(prefix, ".waf", new File(dev.getPath(), waPath));
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+        return dev.createFileReference(waPath + File.separator + waf.getName());
+    }
+
+    private abstract class AsyncRequest implements IIOFuture, Runnable {
+        protected final FileHandle fHandle;
+        protected final long offset;
+        protected final ByteBuffer data;
+        private boolean complete;
+        private HyracksDataException exception;
+        private int result;
+
+        private AsyncRequest(FileHandle fHandle, long offset, ByteBuffer data) {
+            this.fHandle = fHandle;
+            this.offset = offset;
+            this.data = data;
+            complete = false;
+            exception = null;
+        }
+
+        @Override
+        public void run() {
+            HyracksDataException hde = null;
+            int res = -1;
+            try {
+                res = performOperation();
+            } catch (HyracksDataException e) {
+                hde = e;
+            }
+            synchronized (this) {
+                exception = hde;
+                result = res;
+                complete = true;
+                notifyAll();
+            }
+        }
+
+        protected abstract int performOperation() throws HyracksDataException;
+
+        @Override
+        public synchronized int synchronize() throws HyracksDataException, InterruptedException {
+            while (!complete) {
+                wait();
+            }
+            if (exception != null) {
+                throw exception;
+            }
+            return result;
+        }
+
+        @Override
+        public synchronized boolean isComplete() {
+            return complete;
+        }
+    }
+
+    private class AsyncReadRequest extends AsyncRequest {
+        private AsyncReadRequest(FileHandle fHandle, long offset, ByteBuffer data) {
+            super(fHandle, offset, data);
+        }
+
+        @Override
+        protected int performOperation() throws HyracksDataException {
+            return syncRead(fHandle, offset, data);
+        }
+    }
+
+    private class AsyncWriteRequest extends AsyncRequest {
+        private AsyncWriteRequest(FileHandle fHandle, long offset, ByteBuffer data) {
+            super(fHandle, offset, data);
+        }
+
+        @Override
+        protected int performOperation() throws HyracksDataException {
+            return syncWrite(fHandle, offset, data);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/ManagedWorkspaceFileFactory.java b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/ManagedWorkspaceFileFactory.java
new file mode 100644
index 0000000..c6be844
--- /dev/null
+++ b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/ManagedWorkspaceFileFactory.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.control.nc.io;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
+import edu.uci.ics.hyracks.api.resources.IDeallocatable;
+import edu.uci.ics.hyracks.api.resources.IDeallocatableRegistry;
+
+public final class ManagedWorkspaceFileFactory implements IWorkspaceFileFactory {
+    private final IDeallocatableRegistry registry;
+    private final IOManager ioManager;
+
+    public ManagedWorkspaceFileFactory(IDeallocatableRegistry registry, IOManager ioManager) {
+        this.registry = registry;
+        this.ioManager = ioManager;
+    }
+
+    @Override
+    public FileReference createWorkspaceFile(String prefix) throws HyracksDataException {
+        final FileReference fRef = ioManager.createWorkspaceFile(prefix);
+        registry.registerDeallocatable(new IDeallocatable() {
+            @Override
+            public void deallocate() {
+                fRef.getFile().delete();
+            }
+        });
+        return fRef;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/resources/DefaultDeallocatableRegistry.java b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/resources/DefaultDeallocatableRegistry.java
new file mode 100644
index 0000000..863ab7b
--- /dev/null
+++ b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/resources/DefaultDeallocatableRegistry.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.control.nc.resources;
+
+import java.util.List;
+import java.util.Vector;
+
+import edu.uci.ics.hyracks.api.resources.IDeallocatable;
+import edu.uci.ics.hyracks.api.resources.IDeallocatableRegistry;
+
+public class DefaultDeallocatableRegistry implements IDeallocatableRegistry {
+    private final List<IDeallocatable> deallocatables;
+
+    public DefaultDeallocatableRegistry() {
+        deallocatables = new Vector<IDeallocatable>();
+    }
+
+    @Override
+    public void registerDeallocatable(IDeallocatable deallocatable) {
+        deallocatables.add(deallocatable);
+    }
+
+    public void close() {
+        for (IDeallocatable d : deallocatables) {
+            try {
+                d.deallocate();
+            } catch (Exception e) {
+                // ignore
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/runtime/DelegateHyracksContext.java b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/runtime/DelegateHyracksContext.java
deleted file mode 100644
index 963c2e7..0000000
--- a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/runtime/DelegateHyracksContext.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed 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 from
- * 
- *     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 edu.uci.ics.hyracks.control.nc.runtime;
-
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
-import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
-import edu.uci.ics.hyracks.api.resources.IResourceManager;
-
-public class DelegateHyracksContext implements IHyracksContext {
-    private final IHyracksContext delegate;
-
-    private final ICounterContext counterContext;
-
-    private final UUID jobId;
-
-    public DelegateHyracksContext(IHyracksContext delegate, UUID jobId, ICounterContext counterContext) {
-        this.delegate = delegate;
-        this.counterContext = counterContext;
-        this.jobId = jobId;
-    }
-
-    @Override
-    public IResourceManager getResourceManager() {
-        return delegate.getResourceManager();
-    }
-
-    @Override
-    public int getFrameSize() {
-        return delegate.getFrameSize();
-    }
-
-    @Override
-    public UUID getJobId() {
-        return jobId;
-    }
-
-    @Override
-    public ICounterContext getCounterContext() {
-        return counterContext;
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/runtime/OperatorRunnable.java b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/runtime/OperatorRunnable.java
index 6c48ec3..eca7fd0 100644
--- a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/runtime/OperatorRunnable.java
+++ b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/runtime/OperatorRunnable.java
@@ -18,7 +18,7 @@
 
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -29,9 +29,9 @@
     private ByteBuffer buffer;
     private volatile boolean abort;
 
-    public OperatorRunnable(IHyracksContext ctx, IOperatorNodePushable opNode) {
+    public OperatorRunnable(IHyracksStageletContext ctx, IOperatorNodePushable opNode) {
         this.opNode = opNode;
-        buffer = ctx.getResourceManager().allocateFrame();
+        buffer = ctx.allocateFrame();
     }
 
     public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
diff --git a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/runtime/ResourceManager.java b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/runtime/ResourceManager.java
deleted file mode 100644
index 72202e3..0000000
--- a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/runtime/ResourceManager.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed 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 from
- * 
- *     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 edu.uci.ics.hyracks.control.nc.runtime;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
-import edu.uci.ics.hyracks.api.resources.IResourceManager;
-
-public final class ResourceManager implements IResourceManager {
-    private final IHyracksContext ctx;
-
-    public ResourceManager(IHyracksContext ctx) {
-        this.ctx = ctx;
-    }
-
-    @Override
-    public ByteBuffer allocateFrame() {
-        return ByteBuffer.allocate(ctx.getFrameSize());
-    }
-
-    @Override
-    public File createFile(String prefix, String suffix) throws IOException {
-        return File.createTempFile(prefix, suffix);
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/runtime/RootHyracksContext.java b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/runtime/RootHyracksContext.java
index fb7a01d..045fa52 100644
--- a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/runtime/RootHyracksContext.java
+++ b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/runtime/RootHyracksContext.java
@@ -14,24 +14,19 @@
  */
 package edu.uci.ics.hyracks.control.nc.runtime;
 
-import java.util.UUID;
+import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
-import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
-import edu.uci.ics.hyracks.api.resources.IResourceManager;
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.io.IIOManager;
 
-public class RootHyracksContext implements IHyracksContext {
-    private final IResourceManager resourceManager;
+public class RootHyracksContext implements IHyracksRootContext {
     private final int frameSize;
 
-    public RootHyracksContext(int frameSize) {
-        resourceManager = new ResourceManager(this);
-        this.frameSize = frameSize;
-    }
+    private final IIOManager ioManager;
 
-    @Override
-    public IResourceManager getResourceManager() {
-        return resourceManager;
+    public RootHyracksContext(int frameSize, IIOManager ioManager) {
+        this.frameSize = frameSize;
+        this.ioManager = ioManager;
     }
 
     @Override
@@ -40,12 +35,12 @@
     }
 
     @Override
-    public ICounterContext getCounterContext() {
-        throw new UnsupportedOperationException();
+    public IIOManager getIOManager() {
+        return ioManager;
     }
 
     @Override
-    public UUID getJobId() {
-        throw new UnsupportedOperationException();
+    public ByteBuffer allocateFrame() {
+        return ByteBuffer.allocate(frameSize);
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/ConnectionEntry.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/ConnectionEntry.java
deleted file mode 100644
index 4ed4e7f..0000000
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/ConnectionEntry.java
+++ /dev/null
@@ -1,190 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed 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 from
- * 
- *     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 edu.uci.ics.hyracks.dataflow.common.comm;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.SocketChannel;
-import java.util.UUID;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import edu.uci.ics.hyracks.api.comm.IConnectionEntry;
-import edu.uci.ics.hyracks.api.comm.IDataReceiveListener;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
-
-public class ConnectionEntry implements IConnectionEntry {
-    private static final Logger LOGGER = Logger.getLogger(ConnectionEntry.class.getName());
-
-    private SocketChannel socketChannel;
-
-    private final ByteBuffer readBuffer;
-
-    private final ByteBuffer writeBuffer;
-
-    private IDataReceiveListener recvListener;
-
-    private Object attachment;
-
-    private final SelectionKey key;
-
-    private UUID jobId;
-
-    private UUID stageId;
-
-    private boolean aborted;
-
-    public ConnectionEntry(IHyracksContext ctx, SocketChannel socketChannel, SelectionKey key) {
-        this.socketChannel = socketChannel;
-        readBuffer = ctx.getResourceManager().allocateFrame();
-        readBuffer.clear();
-        writeBuffer = ctx.getResourceManager().allocateFrame();
-        writeBuffer.clear();
-        this.key = key;
-    }
-
-    public SocketChannel getSocketChannel() {
-        return socketChannel;
-    }
-
-    public boolean dispatch(SelectionKey key) throws IOException {
-        if (aborted) {
-            recvListener.dataReceived(this);
-        } else {
-            if (key.isReadable()) {
-                if (LOGGER.isLoggable(Level.FINER)) {
-                    LOGGER.finer("Before read: " + readBuffer.position() + " " + readBuffer.limit());
-                }
-                int bytesRead = socketChannel.read(readBuffer);
-                if (bytesRead < 0) {
-                    recvListener.eos(this);
-                    return true;
-                }
-                if (LOGGER.isLoggable(Level.FINER)) {
-                    LOGGER.finer("After read: " + readBuffer.position() + " " + readBuffer.limit());
-                }
-                recvListener.dataReceived(this);
-            } else if (key.isWritable()) {
-                synchronized (this) {
-                    writeBuffer.flip();
-                    if (LOGGER.isLoggable(Level.FINER)) {
-                        LOGGER.finer("Before write: " + writeBuffer.position() + " " + writeBuffer.limit());
-                    }
-                    int bytesWritten = socketChannel.write(writeBuffer);
-                    if (bytesWritten < 0) {
-                        return true;
-                    }
-                    if (LOGGER.isLoggable(Level.FINER)) {
-                        LOGGER.finer("After write: " + writeBuffer.position() + " " + writeBuffer.limit());
-                    }
-                    if (writeBuffer.remaining() <= 0) {
-                        int ops = key.interestOps();
-                        key.interestOps(ops & ~SelectionKey.OP_WRITE);
-                    }
-                    writeBuffer.compact();
-                    notifyAll();
-                }
-            } else {
-                LOGGER.warning("Spurious event triggered: " + key.readyOps());
-                return true;
-            }
-        }
-        return false;
-    }
-
-    @Override
-    public ByteBuffer getReadBuffer() {
-        return readBuffer;
-    }
-
-    @Override
-    public synchronized void write(ByteBuffer buffer) {
-        while (buffer.remaining() > 0) {
-            while (writeBuffer.remaining() <= 0) {
-                try {
-                    wait();
-                } catch (InterruptedException e) {
-                }
-            }
-            int oldLimit = buffer.limit();
-            buffer.limit(Math.min(oldLimit, writeBuffer.remaining()));
-            writeBuffer.put(buffer);
-            buffer.limit(oldLimit);
-            int ops = key.interestOps();
-            key.interestOps(ops | SelectionKey.OP_WRITE);
-            key.selector().wakeup();
-        }
-    }
-
-    @Override
-    public void setDataReceiveListener(IDataReceiveListener listener) {
-        this.recvListener = listener;
-    }
-
-    @Override
-    public void attach(Object attachment) {
-        this.attachment = attachment;
-    }
-
-    @Override
-    public Object getAttachment() {
-        return attachment;
-    }
-
-    @Override
-    public void close() {
-        try {
-            socketChannel.close();
-        } catch (IOException e) {
-            e.printStackTrace();
-        }
-    }
-
-    @Override
-    public SelectionKey getSelectionKey() {
-        return key;
-    }
-
-    @Override
-    public UUID getJobId() {
-        return jobId;
-    }
-
-    @Override
-    public void setJobId(UUID jobId) {
-        this.jobId = jobId;
-    }
-
-    @Override
-    public UUID getStageId() {
-        return stageId;
-    }
-
-    @Override
-    public void setStageId(UUID stageId) {
-        this.stageId = stageId;
-    }
-
-    @Override
-    public void abort() {
-        aborted = true;
-    }
-
-    @Override
-    public boolean aborted() {
-        return aborted;
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/NonDeterministicFrameReader.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/NonDeterministicFrameReader.java
index 3955213..49d3db7 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/NonDeterministicFrameReader.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/NonDeterministicFrameReader.java
@@ -22,18 +22,18 @@
 import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
 import edu.uci.ics.hyracks.api.comm.IConnectionEntry;
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 public class NonDeterministicFrameReader implements IFrameReader {
     private static final Logger LOGGER = Logger.getLogger(NonDeterministicFrameReader.class.getName());
 
-    private final IHyracksContext ctx;
+    private final IHyracksStageletContext ctx;
     private final IConnectionDemultiplexer demux;
     private int lastReadSender;
     private boolean eos;
 
-    public NonDeterministicFrameReader(IHyracksContext ctx, IConnectionDemultiplexer demux) {
+    public NonDeterministicFrameReader(IHyracksStageletContext ctx, IConnectionDemultiplexer demux) {
         this.ctx = ctx;
         this.demux = demux;
     }
@@ -61,7 +61,7 @@
             }
             lastReadSender = (Integer) entry.getAttachment();
             ByteBuffer netBuffer = entry.getReadBuffer();
-            int tupleCount = netBuffer.getInt(FrameHelper.getTupleCountOffset(ctx));
+            int tupleCount = netBuffer.getInt(FrameHelper.getTupleCountOffset(ctx.getFrameSize()));
             if (LOGGER.isLoggable(Level.FINER)) {
                 LOGGER.finer("Frame Tuple Count: " + tupleCount);
             }
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/SortMergeFrameReader.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/SortMergeFrameReader.java
index c86acba..7be9b08 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/SortMergeFrameReader.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/SortMergeFrameReader.java
@@ -14,24 +14,23 @@
  */
 package edu.uci.ics.hyracks.dataflow.common.comm;
 
-import java.io.File;
-import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
 import java.util.Comparator;
 import java.util.PriorityQueue;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
 import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
 import edu.uci.ics.hyracks.api.comm.IConnectionEntry;
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameHelper;
+import edu.uci.ics.hyracks.api.io.FileHandle;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IIOManager;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTuplePairComparator;
@@ -39,7 +38,7 @@
 public class SortMergeFrameReader implements IFrameReader {
     private static final Logger LOGGER = Logger.getLogger(SortMergeFrameReader.class.getName());
 
-    private final IHyracksContext ctx;
+    private final IHyracksStageletContext ctx;
     private final IConnectionDemultiplexer demux;
     private final FrameTuplePairComparator tpc;
     private final FrameTupleAppender appender;
@@ -50,12 +49,12 @@
     private int lastReadSender;
     private boolean first;
 
-    public SortMergeFrameReader(IHyracksContext ctx, IConnectionDemultiplexer demux, int[] sortFields,
+    public SortMergeFrameReader(IHyracksStageletContext ctx, IConnectionDemultiplexer demux, int[] sortFields,
             IBinaryComparator[] comparators, RecordDescriptor recordDescriptor) {
         this.ctx = ctx;
         this.demux = demux;
         tpc = new FrameTuplePairComparator(sortFields, sortFields, comparators);
-        appender = new FrameTupleAppender(ctx);
+        appender = new FrameTupleAppender(ctx.getFrameSize());
         this.recordDescriptor = recordDescriptor;
     }
 
@@ -66,7 +65,7 @@
         frames = new ByteBuffer[nSenders];
         for (int i = 0; i < runs.length; ++i) {
             runs[i] = new Run(i);
-            frames[i] = ctx.getResourceManager().allocateFrame();
+            frames[i] = ctx.allocateFrame();
         }
         pQueue = new PriorityQueue<Integer>(nSenders, new Comparator<Integer>() {
             @Override
@@ -122,8 +121,8 @@
 
     private class Run {
         private final int runId;
-        private final File file;
-        private final FileChannel channel;
+        private final FileReference fRef;
+        private final FileHandle fHandle;
         private final ByteBuffer frame;
         private final FrameTupleAccessor accessor;
         private int tIndex;
@@ -133,43 +132,23 @@
 
         public Run(int runId) throws HyracksDataException {
             this.runId = runId;
-            try {
-                file = ctx.getResourceManager().createFile(SortMergeFrameReader.class.getSimpleName(), ".run");
-                RandomAccessFile raf = new RandomAccessFile(file, "rw");
-                channel = raf.getChannel();
-            } catch (Exception e) {
-                throw new HyracksDataException(e);
-            }
-            frame = ctx.getResourceManager().allocateFrame();
-            accessor = new FrameTupleAccessor(ctx, recordDescriptor);
+            fRef = ctx.createWorkspaceFile(SortMergeFrameReader.class.getName());
+            fHandle = ctx.getIOManager().open(fRef, IIOManager.FileReadWriteMode.READ_WRITE,
+                    IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+            frame = ctx.allocateFrame();
+            accessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
             readFP = 0;
             writeFP = 0;
             eof = false;
         }
 
         public void close() throws HyracksDataException {
-            try {
-                channel.close();
-                file.delete();
-            } catch (IOException e) {
-                throw new HyracksDataException(e);
-            }
+            ctx.getIOManager().close(fHandle);
         }
 
         private void write(ByteBuffer frame) throws HyracksDataException {
-            try {
-                int len = frame.capacity();
-                while (len > 0) {
-                    int sz = channel.write(frame, writeFP);
-                    if (sz < 0) {
-                        throw new HyracksDataException("Error writing to run");
-                    }
-                    len -= sz;
-                    writeFP += sz;
-                }
-            } catch (IOException e) {
-                throw new HyracksDataException(e);
-            }
+            int sz = ctx.getIOManager().syncWrite(fHandle, writeFP, frame);
+            writeFP += sz;
         }
 
         private boolean next() throws HyracksDataException {
@@ -185,28 +164,16 @@
         }
 
         private boolean read(ByteBuffer frame) throws HyracksDataException {
+            frame.clear();
             while (!eof && readFP >= writeFP) {
                 spoolRuns(runId);
             }
             if (eof && readFP >= writeFP) {
                 return false;
             }
-            try {
-                channel.position(readFP);
-                frame.clear();
-                int len = frame.capacity();
-                while (len > 0) {
-                    int sz = channel.read(frame, readFP);
-                    if (sz < 0) {
-                        throw new HyracksDataException("Error reading file");
-                    }
-                    len -= sz;
-                    readFP += sz;
-                }
-                return true;
-            } catch (IOException e) {
-                throw new HyracksDataException(e);
-            }
+            int sz = ctx.getIOManager().syncRead(fHandle, readFP, frame);
+            readFP += sz;
+            return true;
         }
 
         private void eof() {
@@ -219,7 +186,7 @@
             IConnectionEntry entry = demux.findNextReadyEntry(lastReadSender);
             lastReadSender = (Integer) entry.getAttachment();
             ByteBuffer netBuffer = entry.getReadBuffer();
-            int tupleCount = netBuffer.getInt(FrameHelper.getTupleCountOffset(ctx));
+            int tupleCount = netBuffer.getInt(FrameHelper.getTupleCountOffset(ctx.getFrameSize()));
             if (LOGGER.isLoggable(Level.FINER)) {
                 LOGGER.finer("Frame Tuple Count: " + tupleCount);
             }
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializer.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializer.java
index 8c6b7c9..61638d1 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializer.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializer.java
@@ -20,7 +20,6 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
@@ -42,11 +41,11 @@
 
     private ByteBuffer buffer;
 
-    public FrameDeserializer(IHyracksContext ctx, RecordDescriptor recordDescriptor) {
+    public FrameDeserializer(int frameSize, RecordDescriptor recordDescriptor) {
         this.bbis = new ByteBufferInputStream();
         this.di = new DataInputStream(bbis);
         this.recordDescriptor = recordDescriptor;
-        frameTupleAccessor = new FrameTupleAccessor(ctx, recordDescriptor);
+        frameTupleAccessor = new FrameTupleAccessor(frameSize, recordDescriptor);
     }
 
     public void reset(ByteBuffer buffer) {
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java
index f18dacd..d2a3c53 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java
@@ -17,7 +17,7 @@
 import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOpenableDataReader;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -33,10 +33,11 @@
 
     private final FrameDeserializer frameDeserializer;
 
-    public FrameDeserializingDataReader(IHyracksContext ctx, IFrameReader frameReader, RecordDescriptor recordDescriptor) {
-        buffer = ctx.getResourceManager().allocateFrame();
+    public FrameDeserializingDataReader(IHyracksStageletContext ctx, IFrameReader frameReader,
+            RecordDescriptor recordDescriptor) {
+        buffer = ctx.allocateFrame();
         this.frameReader = frameReader;
-        this.frameDeserializer = new FrameDeserializer(ctx, recordDescriptor);
+        this.frameDeserializer = new FrameDeserializer(ctx.getFrameSize(), recordDescriptor);
     }
 
     @Override
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataWriter.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataWriter.java
index 0a56bea..96a8102 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataWriter.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataWriter.java
@@ -17,7 +17,7 @@
 import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
 import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -26,10 +26,10 @@
     private final IOpenableDataWriter<Object[]> writer;
     private final FrameDeserializer frameDeserializer;
 
-    public FrameDeserializingDataWriter(IHyracksContext ctx, IOpenableDataWriter<Object[]> writer,
+    public FrameDeserializingDataWriter(IHyracksRootContext ctx, IOpenableDataWriter<Object[]> writer,
             RecordDescriptor recordDescriptor) {
         this.writer = writer;
-        this.frameDeserializer = new FrameDeserializer(ctx, recordDescriptor);
+        this.frameDeserializer = new FrameDeserializer(ctx.getFrameSize(), recordDescriptor);
     }
 
     @Override
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java
index e35c54b..c132032 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java
@@ -17,8 +17,8 @@
 import java.io.DataInputStream;
 import java.nio.ByteBuffer;
 
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
@@ -34,13 +34,13 @@
  * @author vinayakb
  */
 public final class FrameTupleAccessor implements IFrameTupleAccessor {
-    private final IHyracksContext ctx;
+    private final int frameSize;
     private final RecordDescriptor recordDescriptor;
 
     private ByteBuffer buffer;
 
-    public FrameTupleAccessor(IHyracksContext ctx, RecordDescriptor recordDescriptor) {
-        this.ctx = ctx;
+    public FrameTupleAccessor(int frameSize, RecordDescriptor recordDescriptor) {
+        this.frameSize = frameSize;
         this.recordDescriptor = recordDescriptor;
     }
 
@@ -56,17 +56,17 @@
 
     @Override
     public int getTupleCount() {
-        return buffer.getInt(FrameHelper.getTupleCountOffset(ctx));
+        return buffer.getInt(FrameHelper.getTupleCountOffset(frameSize));
     }
 
     @Override
     public int getTupleStartOffset(int tupleIndex) {
-        return tupleIndex == 0 ? 0 : buffer.getInt(FrameHelper.getTupleCountOffset(ctx) - 4 * tupleIndex);
+        return tupleIndex == 0 ? 0 : buffer.getInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * tupleIndex);
     }
 
     @Override
     public int getTupleEndOffset(int tupleIndex) {
-        return buffer.getInt(FrameHelper.getTupleCountOffset(ctx) - 4 * (tupleIndex + 1));
+        return buffer.getInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleIndex + 1));
     }
 
     @Override
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppender.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppender.java
index bd5a907..2e9daf3 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppender.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppender.java
@@ -16,11 +16,11 @@
 
 import java.nio.ByteBuffer;
 
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
 
 public class FrameTupleAppender {
-    private final IHyracksContext ctx;
+    private final int frameSize;
 
     private ByteBuffer buffer;
 
@@ -28,33 +28,33 @@
 
     private int tupleDataEndOffset;
 
-    public FrameTupleAppender(IHyracksContext ctx) {
-        this.ctx = ctx;
+    public FrameTupleAppender(int frameSize) {
+        this.frameSize = frameSize;
     }
 
     public void reset(ByteBuffer buffer, boolean clear) {
         this.buffer = buffer;
         if (clear) {
-            buffer.putInt(FrameHelper.getTupleCountOffset(ctx), 0);
+            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), 0);
             tupleCount = 0;
             tupleDataEndOffset = 0;
         } else {
-            tupleCount = buffer.getInt(FrameHelper.getTupleCountOffset(ctx));
-            tupleDataEndOffset = tupleCount == 0 ? 0 : buffer.getInt(FrameHelper.getTupleCountOffset(ctx) - tupleCount
-                    * 4);
+            tupleCount = buffer.getInt(FrameHelper.getTupleCountOffset(frameSize));
+            tupleDataEndOffset = tupleCount == 0 ? 0 : buffer.getInt(FrameHelper.getTupleCountOffset(frameSize)
+                    - tupleCount * 4);
         }
     }
 
     public boolean append(int[] fieldSlots, byte[] bytes, int offset, int length) {
-        if (tupleDataEndOffset + fieldSlots.length * 2 + length + 4 + (tupleCount + 1) * 4 <= ctx.getFrameSize()) {
+        if (tupleDataEndOffset + fieldSlots.length * 2 + length + 4 + (tupleCount + 1) * 4 <= frameSize) {
             for (int i = 0; i < fieldSlots.length; ++i) {
                 buffer.putShort(tupleDataEndOffset + i * 2, (short) fieldSlots[i]);
             }
             System.arraycopy(bytes, offset, buffer.array(), tupleDataEndOffset + fieldSlots.length * 2, length);
             tupleDataEndOffset += fieldSlots.length * 2 + length;
-            buffer.putInt(FrameHelper.getTupleCountOffset(ctx) - 4 * (tupleCount + 1), tupleDataEndOffset);
+            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleCount + 1), tupleDataEndOffset);
             ++tupleCount;
-            buffer.putInt(FrameHelper.getTupleCountOffset(ctx), tupleCount);
+            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), tupleCount);
             return true;
         }
         return false;
@@ -62,13 +62,13 @@
 
     public boolean append(IFrameTupleAccessor tupleAccessor, int tStartOffset, int tEndOffset) {
         int length = tEndOffset - tStartOffset;
-        if (tupleDataEndOffset + length + 4 + (tupleCount + 1) * 4 <= ctx.getFrameSize()) {
+        if (tupleDataEndOffset + length + 4 + (tupleCount + 1) * 4 <= frameSize) {
             ByteBuffer src = tupleAccessor.getBuffer();
             System.arraycopy(src.array(), tStartOffset, buffer.array(), tupleDataEndOffset, length);
             tupleDataEndOffset += length;
-            buffer.putInt(FrameHelper.getTupleCountOffset(ctx) - 4 * (tupleCount + 1), tupleDataEndOffset);
+            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleCount + 1), tupleDataEndOffset);
             ++tupleCount;
-            buffer.putInt(FrameHelper.getTupleCountOffset(ctx), tupleCount);
+            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), tupleCount);
             return true;
         }
         return false;
@@ -89,7 +89,7 @@
         int endOffset1 = accessor1.getTupleEndOffset(tIndex1);
         int length1 = endOffset1 - startOffset1;
 
-        if (tupleDataEndOffset + length0 + length1 + 4 + (tupleCount + 1) * 4 <= ctx.getFrameSize()) {
+        if (tupleDataEndOffset + length0 + length1 + 4 + (tupleCount + 1) * 4 <= frameSize) {
             ByteBuffer src0 = accessor0.getBuffer();
             ByteBuffer src1 = accessor1.getBuffer();
             int slotsLen0 = accessor0.getFieldSlotsLength();
@@ -110,9 +110,9 @@
             System.arraycopy(src1.array(), startOffset1 + slotsLen1, buffer.array(), tupleDataEndOffset + slotsLen0
                     + slotsLen1 + dataLen0, dataLen1);
             tupleDataEndOffset += (length0 + length1);
-            buffer.putInt(FrameHelper.getTupleCountOffset(ctx) - 4 * (tupleCount + 1), tupleDataEndOffset);
+            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleCount + 1), tupleDataEndOffset);
             ++tupleCount;
-            buffer.putInt(FrameHelper.getTupleCountOffset(ctx), tupleCount);
+            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), tupleCount);
             return true;
         }
         return false;
@@ -125,7 +125,7 @@
             length += (accessor.getFieldEndOffset(tIndex, fields[i]) - accessor.getFieldStartOffset(tIndex, fields[i]));
         }
 
-        if (tupleDataEndOffset + length + 4 + (tupleCount + 1) * 4 <= ctx.getFrameSize()) {
+        if (tupleDataEndOffset + length + 4 + (tupleCount + 1) * 4 <= frameSize) {
             int fSrcSlotsLength = accessor.getFieldSlotsLength();
             int tStartOffset = accessor.getTupleStartOffset(tIndex);
 
@@ -142,9 +142,9 @@
                 fStartOffset = fEndOffset;
             }
             tupleDataEndOffset += length;
-            buffer.putInt(FrameHelper.getTupleCountOffset(ctx) - 4 * (tupleCount + 1), tupleDataEndOffset);
+            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleCount + 1), tupleDataEndOffset);
             ++tupleCount;
-            buffer.putInt(FrameHelper.getTupleCountOffset(ctx), tupleCount);
+            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), tupleCount);
             return true;
         }
         return false;
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
index 078e4c0..9edd3bf 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
@@ -19,7 +19,7 @@
 import java.util.logging.Logger;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -39,12 +39,13 @@
 
     private boolean open;
 
-    public SerializingDataWriter(IHyracksContext ctx, RecordDescriptor recordDescriptor, IFrameWriter frameWriter) {
-        buffer = ctx.getResourceManager().allocateFrame();
+    public SerializingDataWriter(IHyracksStageletContext ctx, RecordDescriptor recordDescriptor,
+            IFrameWriter frameWriter) {
+        buffer = ctx.allocateFrame();
         tb = new ArrayTupleBuilder(recordDescriptor.getFields().length);
         this.recordDescriptor = recordDescriptor;
         this.frameWriter = frameWriter;
-        tupleAppender = new FrameTupleAppender(ctx);
+        tupleAppender = new FrameTupleAppender(ctx.getFrameSize());
         open = false;
     }
 
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/io/RunFileReader.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/io/RunFileReader.java
new file mode 100644
index 0000000..063d0f7
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/io/RunFileReader.java
@@ -0,0 +1,45 @@
+package edu.uci.ics.hyracks.dataflow.common.io;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileHandle;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+
+public class RunFileReader implements IFrameReader {
+    private final FileReference file;
+    private final IIOManager ioManager;
+    private final long size;
+
+    private FileHandle handle;
+    private long readPtr;
+
+    public RunFileReader(FileReference file, IIOManager ioManager, long size) {
+        this.file = file;
+        this.ioManager = ioManager;
+        this.size = size;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        handle = ioManager.open(file, IIOManager.FileReadWriteMode.READ_ONLY, null);
+        readPtr = 0;
+    }
+
+    @Override
+    public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        buffer.clear();
+        if (readPtr >= size) {
+            return false;
+        }
+        readPtr += ioManager.syncRead(handle, readPtr, buffer);
+        return true;
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        ioManager.close(handle);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/io/RunFileWriter.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/io/RunFileWriter.java
new file mode 100644
index 0000000..8d1ec1d
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/io/RunFileWriter.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.common.io;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileHandle;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+
+public class RunFileWriter implements IFrameWriter {
+    private final FileReference file;
+    private final IIOManager ioManager;
+
+    private FileHandle handle;
+    private long size;
+
+    public RunFileWriter(FileReference file, IIOManager ioManager) {
+        this.file = file;
+        this.ioManager = ioManager;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        handle = ioManager.open(file, IIOManager.FileReadWriteMode.READ_WRITE,
+                IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+        size = 0;
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        size += ioManager.syncWrite(handle, size, buffer);
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        ioManager.close(handle);
+    }
+
+    public FileReference getFileReference() {
+        return file;
+    }
+
+    public long getFileSize() {
+        return size;
+    }
+
+    @Override
+    public void flush() throws HyracksDataException {
+    }
+
+    public RunFileReader createReader() {
+        return new RunFileReader(file, ioManager, size);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopMapperOperatorDescriptor.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopMapperOperatorDescriptor.java
index a216ac9..c3167cb 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopMapperOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopMapperOperatorDescriptor.java
@@ -34,7 +34,7 @@
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.util.ReflectionUtils;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -167,10 +167,10 @@
                 } else if (splitRead instanceof org.apache.hadoop.mapreduce.lib.input.FileSplit) {
                     conf.set("map.input.file", ((org.apache.hadoop.mapreduce.lib.input.FileSplit) splitRead).getPath()
                             .toString());
-                    conf.setLong("map.input.start", ((org.apache.hadoop.mapreduce.lib.input.FileSplit) splitRead)
-                            .getStart());
-                    conf.setLong("map.input.length", ((org.apache.hadoop.mapreduce.lib.input.FileSplit) splitRead)
-                            .getLength());
+                    conf.setLong("map.input.start",
+                            ((org.apache.hadoop.mapreduce.lib.input.FileSplit) splitRead).getStart());
+                    conf.setLong("map.input.length",
+                            ((org.apache.hadoop.mapreduce.lib.input.FileSplit) splitRead).getLength());
                 }
             } catch (Exception e) {
                 e.printStackTrace();
@@ -298,9 +298,9 @@
         String mapOutputValueClassName = conf.getMapOutputValueClass().getName();
         try {
             if (hadoopClassFactory == null) {
-                recordDescriptor = DatatypeHelper.createKeyValueRecordDescriptor((Class<? extends Writable>) Class
-                        .forName(mapOutputKeyClassName), (Class<? extends Writable>) Class
-                        .forName(mapOutputValueClassName));
+                recordDescriptor = DatatypeHelper.createKeyValueRecordDescriptor(
+                        (Class<? extends Writable>) Class.forName(mapOutputKeyClassName),
+                        (Class<? extends Writable>) Class.forName(mapOutputValueClassName));
             } else {
                 recordDescriptor = DatatypeHelper.createKeyValueRecordDescriptor(
                         (Class<? extends Writable>) hadoopClassFactory.loadClass(mapOutputKeyClassName),
@@ -326,7 +326,7 @@
                 mapperClass = super.getJobConf().getMapperClass();
                 mapperClassName = mapperClass.getName();
             }
-            mapper = getHadoopClassFactory().createMapper(mapperClassName,jobConf);
+            mapper = getHadoopClassFactory().createMapper(mapperClassName, jobConf);
         }
         return mapper;
     }
@@ -344,13 +344,13 @@
         } else {
             Class inputFormatClass = conf.getInputFormat().getClass();
             InputFormat inputFormat = (InputFormat) ReflectionUtils.newInstance(inputFormatClass, conf);
-            return inputFormat.getRecordReader((org.apache.hadoop.mapred.InputSplit) inputSplit, conf, super
-                    .createReporter());
+            return inputFormat.getRecordReader((org.apache.hadoop.mapred.InputSplit) inputSplit, conf,
+                    super.createReporter());
         }
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
 
         JobConf conf = getJobConf();
@@ -383,15 +383,15 @@
                 }
                 return createSelfReadingMapper(ctx, env, recordDescriptor, partition);
             } else {
-                return new DeserializedOperatorNodePushable(ctx, new MapperOperator(partition), recordDescProvider
-                        .getInputRecordDescriptor(this.odId, 0));
+                return new DeserializedOperatorNodePushable(ctx, new MapperOperator(partition),
+                        recordDescProvider.getInputRecordDescriptor(this.odId, 0));
             }
         } catch (Exception e) {
             throw new HyracksDataException(e);
         }
     }
 
-    private IOperatorNodePushable createSelfReadingMapper(final IHyracksContext ctx, IOperatorEnvironment env,
+    private IOperatorNodePushable createSelfReadingMapper(final IHyracksStageletContext ctx, IOperatorEnvironment env,
             final RecordDescriptor recordDescriptor, final int partition) {
         return new AbstractUnaryOutputSourceOperatorNodePushable() {
             @Override
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
index 93ef1ec..4f0ae2f 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
@@ -20,19 +20,19 @@
 import java.util.Map;
 
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.Counters.Counter;
 import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.SequenceFileRecordReader;
-import org.apache.hadoop.mapred.Counters.Counter;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.util.ReflectionUtils;
 
 import edu.uci.ics.hyracks.api.constraints.PartitionCountConstraint;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -71,29 +71,33 @@
             recordReader = getRecordReader(DatatypeHelper.map2JobConf(jobConfMap), splits[0]);
         } catch (Exception e) {
             throw new IOException(e);
-        } 
+        }
         recordDescriptors[0] = DatatypeHelper.createKeyValueRecordDescriptor((Class<? extends Writable>) recordReader
                 .createKey().getClass(), (Class<? extends Writable>) recordReader.createValue().getClass());
         this.setPartitionConstraint(new PartitionCountConstraint(splits.length));
-        inputSplitsProxy = new InputSplitsProxy(jobConf,splits);
+        inputSplitsProxy = new InputSplitsProxy(jobConf, splits);
         this.inputFormatClassName = inputFormat.getClass().getName();
     }
 
-    private RecordReader getRecordReader(JobConf conf, Object inputSplit) throws ClassNotFoundException, IOException, InterruptedException {
+    private RecordReader getRecordReader(JobConf conf, Object inputSplit) throws ClassNotFoundException, IOException,
+            InterruptedException {
         RecordReader hadoopRecordReader = null;
-        if(conf.getUseNewMapper()){
-            JobContext context = new JobContext(conf,null);
-            org.apache.hadoop.mapreduce.InputFormat inputFormat = (org.apache.hadoop.mapreduce.InputFormat) ReflectionUtils.newInstance(context.getInputFormatClass(), conf);
+        if (conf.getUseNewMapper()) {
+            JobContext context = new JobContext(conf, null);
+            org.apache.hadoop.mapreduce.InputFormat inputFormat = (org.apache.hadoop.mapreduce.InputFormat) ReflectionUtils
+                    .newInstance(context.getInputFormatClass(), conf);
             TaskAttemptContext taskAttemptContext = new org.apache.hadoop.mapreduce.TaskAttemptContext(jobConf, null);
-            hadoopRecordReader = (RecordReader) inputFormat.createRecordReader((org.apache.hadoop.mapreduce.InputSplit)inputSplit,taskAttemptContext);
+            hadoopRecordReader = (RecordReader) inputFormat.createRecordReader(
+                    (org.apache.hadoop.mapreduce.InputSplit) inputSplit, taskAttemptContext);
         } else {
             Class inputFormatClass = conf.getInputFormat().getClass();
             InputFormat inputFormat = (InputFormat) ReflectionUtils.newInstance(inputFormatClass, conf);
-            hadoopRecordReader = (RecordReader)inputFormat.getRecordReader((org.apache.hadoop.mapred.InputSplit)inputSplit, conf, createReporter());
+            hadoopRecordReader = (RecordReader) inputFormat.getRecordReader(
+                    (org.apache.hadoop.mapred.InputSplit) inputSplit, conf, createReporter());
         }
         return hadoopRecordReader;
     }
-    
+
     public Object[] getInputSplits() throws InstantiationException, IllegalAccessException, IOException {
         return inputSplitsProxy.toInputSplits(getJobConf());
     }
@@ -139,7 +143,7 @@
 
     @SuppressWarnings("deprecation")
     @Override
-    public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx, IOperatorEnvironment env,
             final IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
             throws HyracksDataException {
         return new AbstractUnaryOutputSourceOperatorNodePushable() {
@@ -154,18 +158,22 @@
                     Object value;
                     Object[] splits = inputSplitsProxy.toInputSplits(conf);
                     Object inputSplit = splits[partition];
-                    
-                    if(conf.getUseNewMapper()){
-                        JobContext context = new JobContext(conf,null);
-                        org.apache.hadoop.mapreduce.InputFormat inputFormat = (org.apache.hadoop.mapreduce.InputFormat) ReflectionUtils.newInstance(context.getInputFormatClass(), conf);
-                        TaskAttemptContext taskAttemptContext = new org.apache.hadoop.mapreduce.TaskAttemptContext(jobConf, null);
-                        hadoopRecordReader = (RecordReader) inputFormat.createRecordReader((org.apache.hadoop.mapreduce.InputSplit)inputSplit,taskAttemptContext);
+
+                    if (conf.getUseNewMapper()) {
+                        JobContext context = new JobContext(conf, null);
+                        org.apache.hadoop.mapreduce.InputFormat inputFormat = (org.apache.hadoop.mapreduce.InputFormat) ReflectionUtils
+                                .newInstance(context.getInputFormatClass(), conf);
+                        TaskAttemptContext taskAttemptContext = new org.apache.hadoop.mapreduce.TaskAttemptContext(
+                                jobConf, null);
+                        hadoopRecordReader = (RecordReader) inputFormat.createRecordReader(
+                                (org.apache.hadoop.mapreduce.InputSplit) inputSplit, taskAttemptContext);
                     } else {
                         Class inputFormatClass = conf.getInputFormat().getClass();
                         InputFormat inputFormat = (InputFormat) ReflectionUtils.newInstance(inputFormatClass, conf);
-                        hadoopRecordReader = (RecordReader)inputFormat.getRecordReader((org.apache.hadoop.mapred.InputSplit)inputSplit, conf, createReporter());
+                        hadoopRecordReader = (RecordReader) inputFormat.getRecordReader(
+                                (org.apache.hadoop.mapred.InputSplit) inputSplit, conf, createReporter());
                     }
-                    
+
                     Class inputKeyClass;
                     Class inputValueClass;
                     if (hadoopRecordReader instanceof SequenceFileRecordReader) {
@@ -178,8 +186,8 @@
 
                     key = hadoopRecordReader.createKey();
                     value = hadoopRecordReader.createValue();
-                    ByteBuffer outBuffer = ctx.getResourceManager().allocateFrame();
-                    FrameTupleAppender appender = new FrameTupleAppender(ctx);
+                    ByteBuffer outBuffer = ctx.allocateFrame();
+                    FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
                     appender.reset(outBuffer, true);
                     RecordDescriptor outputRecordDescriptor = DatatypeHelper.createKeyValueRecordDescriptor(
                             (Class<? extends Writable>) hadoopRecordReader.createKey().getClass(),
@@ -213,7 +221,7 @@
                     throw new HyracksDataException(e);
                 } catch (ClassNotFoundException e) {
                     throw new HyracksDataException(e);
-                } catch (InterruptedException e){
+                } catch (InterruptedException e) {
                     throw new HyracksDataException(e);
                 } catch (IOException e) {
                     throw new HyracksDataException(e);
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReducerOperatorDescriptor.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReducerOperatorDescriptor.java
index 265147f..2d44c79 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReducerOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReducerOperatorDescriptor.java
@@ -23,19 +23,18 @@
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.mapred.Counters.Counter;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RawKeyValueIterator;
 import org.apache.hadoop.mapred.Reducer;
 import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.Counters.Counter;
 import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.StatusReporter;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.util.Progress;
 import org.apache.hadoop.util.ReflectionUtils;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IDataReader;
 import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
@@ -60,64 +59,66 @@
         private Object reducer;
         private DataWritingOutputCollector<K3, V3> output;
         private Reporter reporter;
-        private ReducerContext reducerContext; 
+        private ReducerContext reducerContext;
         RawKeyValueIterator rawKeyValueIterator = new RawKeyValueIterator() {
-            
+
             @Override
             public boolean next() throws IOException {
                 return false;
             }
-            
+
             @Override
             public DataInputBuffer getValue() throws IOException {
                 return null;
             }
-            
+
             @Override
             public Progress getProgress() {
                 return null;
             }
-            
+
             @Override
             public DataInputBuffer getKey() throws IOException {
                 return null;
             }
-            
+
             @Override
             public void close() throws IOException {
-                
+
             }
         };
-        
-    
+
         class ReducerContext extends org.apache.hadoop.mapreduce.Reducer.Context {
             private HadoopReducerOperatorDescriptor.ValueIterator iterator;
-            
+
             @SuppressWarnings("unchecked")
-            ReducerContext(org.apache.hadoop.mapreduce.Reducer reducer, JobConf conf) throws IOException, InterruptedException, ClassNotFoundException{
-            
-                reducer.super(conf,new TaskAttemptID(),rawKeyValueIterator,null,null,null,null,null,null,Class.forName("org.apache.hadoop.io.NullWritable"),Class.forName("org.apache.hadoop.io.NullWritable"));
+            ReducerContext(org.apache.hadoop.mapreduce.Reducer reducer, JobConf conf) throws IOException,
+                    InterruptedException, ClassNotFoundException {
+
+                reducer.super(conf, new TaskAttemptID(), rawKeyValueIterator, null, null, null, null, null, null, Class
+                        .forName("org.apache.hadoop.io.NullWritable"), Class
+                        .forName("org.apache.hadoop.io.NullWritable"));
             }
-            
-            public  void setIterator(HadoopReducerOperatorDescriptor.ValueIterator iter) {
+
+            public void setIterator(HadoopReducerOperatorDescriptor.ValueIterator iter) {
                 iterator = iter;
             }
-            
+
             @Override
             public Iterable<V2> getValues() throws IOException, InterruptedException {
-              return new Iterable<V2>() {
-                @Override
-                public Iterator<V2> iterator() {
-                    return iterator;
-                }
-              };
+                return new Iterable<V2>() {
+                    @Override
+                    public Iterator<V2> iterator() {
+                        return iterator;
+                    }
+                };
             }
-            
+
             /** Start processing next unique key. */
             @Override
-            public boolean nextKey() throws IOException,InterruptedException {
+            public boolean nextKey() throws IOException, InterruptedException {
                 boolean hasMore = iterator.hasNext();
-                if(hasMore){
+                if (hasMore) {
                     nextKeyValue();
                 }
                 return hasMore;
@@ -133,26 +134,25 @@
             }
 
             public Object getCurrentKey() {
-              return iterator.getKey();
+                return iterator.getKey();
             }
 
             @Override
             public Object getCurrentValue() {
-              return iterator.getValue();
+                return iterator.getValue();
             }
-            
+
             /**
              * Generate an output key/value pair.
              */
             @Override
-            public void write(Object key, Object value
-                              ) throws IOException, InterruptedException {
-              output.collect(key, value);
+            public void write(Object key, Object value) throws IOException, InterruptedException {
+                output.collect(key, value);
             }
 
         }
-            
-        public ReducerAggregator(Object reducer) throws HyracksDataException{
+
+        public ReducerAggregator(Object reducer) throws HyracksDataException {
             this.reducer = reducer;
             initializeReducer();
             output = new DataWritingOutputCollector<K3, V3>();
@@ -201,17 +201,17 @@
             i.reset(reader);
             output.setWriter(writer);
             try {
-                if(jobConf.getUseNewReducer()){
+                if (jobConf.getUseNewReducer()) {
                     try {
                         reducerContext.setIterator(i);
-                        ((org.apache.hadoop.mapreduce.Reducer)reducer).run(reducerContext);
+                        ((org.apache.hadoop.mapreduce.Reducer) reducer).run(reducerContext);
                     } catch (InterruptedException e) {
                         e.printStackTrace();
                         throw new HyracksDataException(e);
                     }
-                } else {  
-                    ((org.apache.hadoop.mapred.Reducer)reducer).reduce(i.getKey(), i, output, reporter);
-                }    
+                } else {
+                    ((org.apache.hadoop.mapred.Reducer) reducer).reduce(i.getKey(), i, output, reporter);
+                }
             } catch (IOException e) {
                 e.printStackTrace();
             }
@@ -221,28 +221,28 @@
         public void close() throws HyracksDataException {
             // -- - close - --
             try {
-                if(!jobConf.getUseNewMapper()) {
-                    ((org.apache.hadoop.mapred.Reducer)reducer).close();
-                }    
+                if (!jobConf.getUseNewMapper()) {
+                    ((org.apache.hadoop.mapred.Reducer) reducer).close();
+                }
             } catch (IOException e) {
                 throw new HyracksDataException(e);
             }
         }
-        
+
         private void initializeReducer() throws HyracksDataException {
             jobConf.setClassLoader(this.getClass().getClassLoader());
-            if(!jobConf.getUseNewReducer()) {
-                ((org.apache.hadoop.mapred.Reducer)reducer).configure(getJobConf());    
+            if (!jobConf.getUseNewReducer()) {
+                ((org.apache.hadoop.mapred.Reducer) reducer).configure(getJobConf());
             } else {
                 try {
-                    reducerContext = new ReducerContext((org.apache.hadoop.mapreduce.Reducer)reducer,jobConf);
+                    reducerContext = new ReducerContext((org.apache.hadoop.mapreduce.Reducer) reducer, jobConf);
                 } catch (IOException e) {
                     e.printStackTrace();
                     throw new HyracksDataException(e);
                 } catch (InterruptedException e) {
                     e.printStackTrace();
                     throw new HyracksDataException(e);
-                } catch (RuntimeException e){
+                } catch (RuntimeException e) {
                     e.printStackTrace();
                 } catch (ClassNotFoundException e) {
                     e.printStackTrace();
@@ -259,7 +259,7 @@
         public K2 getKey() {
             return key;
         }
-        
+
         public V2 getValue() {
             return value;
         }
@@ -322,19 +322,20 @@
             return ReflectionUtils.newInstance(reducerClass, getJobConf());
         } else {
             Object reducer;
-            if(getJobConf().getUseNewReducer()){
+            if (getJobConf().getUseNewReducer()) {
                 JobContext jobContext = new JobContext(getJobConf(), null);
-                reducerClass = (Class<? extends org.apache.hadoop.mapreduce.Reducer<?,?,?,?>> )jobContext.getReducerClass();
+                reducerClass = (Class<? extends org.apache.hadoop.mapreduce.Reducer<?, ?, ?, ?>>) jobContext
+                        .getReducerClass();
             } else {
                 reducerClass = (Class<? extends Reducer>) getJobConf().getReducerClass();
             }
-            reducer = getHadoopClassFactory().createReducer(reducerClass.getName(),getJobConf());
+            reducer = getHadoopClassFactory().createReducer(reducerClass.getName(), getJobConf());
             return reducer;
         }
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         try {
             if (this.comparatorFactory == null) {
@@ -366,23 +367,24 @@
     }
 
     public static RecordDescriptor getRecordDescriptor(JobConf conf, IHadoopClassFactory classFactory) {
-        String outputKeyClassName =null; 
+        String outputKeyClassName = null;
         String outputValueClassName = null;
-        
-        if(conf.getUseNewMapper()) {
-            JobContext context = new JobContext(conf,null);
+
+        if (conf.getUseNewMapper()) {
+            JobContext context = new JobContext(conf, null);
             outputKeyClassName = context.getOutputKeyClass().getName();
             outputValueClassName = context.getOutputValueClass().getName();
         } else {
             outputKeyClassName = conf.getOutputKeyClass().getName();
             outputValueClassName = conf.getOutputValueClass().getName();
         }
-        
+
         RecordDescriptor recordDescriptor = null;
         try {
             if (classFactory == null) {
-                recordDescriptor = DatatypeHelper.createKeyValueRecordDescriptor((Class<? extends Writable>) Class
-                        .forName(outputKeyClassName), (Class<? extends Writable>) Class.forName(outputValueClassName));
+                recordDescriptor = DatatypeHelper.createKeyValueRecordDescriptor(
+                        (Class<? extends Writable>) Class.forName(outputKeyClassName),
+                        (Class<? extends Writable>) Class.forName(outputValueClassName));
             } else {
                 recordDescriptor = DatatypeHelper.createKeyValueRecordDescriptor(
                         (Class<? extends Writable>) classFactory.loadClass(outputKeyClassName),
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopWriteOperatorDescriptor.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopWriteOperatorDescriptor.java
index 95a4647..4dcfa61 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopWriteOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopWriteOperatorDescriptor.java
@@ -21,14 +21,12 @@
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.SequenceFile.Writer;
+import org.apache.hadoop.mapred.Counters.Counter;
 import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.SequenceFileOutputFormat;
-import org.apache.hadoop.mapred.Counters.Counter;
 import org.apache.hadoop.mapred.lib.NullOutputFormat;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
@@ -36,6 +34,7 @@
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
 
+import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.hadoop.util.DatatypeHelper;
 import edu.uci.ics.hyracks.dataflow.std.file.AbstractFileWriteOperatorDescriptor;
@@ -44,15 +43,14 @@
 
 public class HadoopWriteOperatorDescriptor extends AbstractFileWriteOperatorDescriptor {
 
-    private  class HadoopFileWriter implements IRecordWriter {
+    private class HadoopFileWriter implements IRecordWriter {
 
         Object recordWriter;
         JobConf conf;
         Path finalOutputFile;
         Path tempOutputFile;
-      
-        
-        HadoopFileWriter(Object recordWriter,Path tempOutputFile,Path outputFile,JobConf conf) {
+
+        HadoopFileWriter(Object recordWriter, Path tempOutputFile, Path outputFile, JobConf conf) {
             this.recordWriter = recordWriter;
             this.conf = conf;
             this.finalOutputFile = outputFile;
@@ -61,22 +59,23 @@
 
         @Override
         public void write(Object[] record) throws Exception {
-            if (conf.getUseNewMapper()){
-                ((org.apache.hadoop.mapreduce.RecordWriter)recordWriter).write(record[0], record[1]);
+            if (conf.getUseNewMapper()) {
+                ((org.apache.hadoop.mapreduce.RecordWriter) recordWriter).write(record[0], record[1]);
             } else {
-                ((org.apache.hadoop.mapred.RecordWriter)recordWriter).write(record[0], record[1]);
-            }    
+                ((org.apache.hadoop.mapred.RecordWriter) recordWriter).write(record[0], record[1]);
+            }
         }
 
         @Override
         public void close() {
             try {
-                if (conf.getUseNewMapper()){
-                   ((org.apache.hadoop.mapreduce.RecordWriter)recordWriter).close(new TaskAttemptContext(conf, new TaskAttemptID()));
+                if (conf.getUseNewMapper()) {
+                    ((org.apache.hadoop.mapreduce.RecordWriter) recordWriter).close(new TaskAttemptContext(conf,
+                            new TaskAttemptID()));
                 } else {
-                    ((org.apache.hadoop.mapred.RecordWriter)recordWriter).close(null);
+                    ((org.apache.hadoop.mapred.RecordWriter) recordWriter).close(null);
                 }
-                FileSystem.get(conf).rename( tempOutputFile, finalOutputFile);
+                FileSystem.get(conf).rename(tempOutputFile, finalOutputFile);
             } catch (Exception e) {
                 e.printStackTrace();
             }
@@ -121,90 +120,90 @@
         } catch (IOException ioe) {
             ioe.printStackTrace();
         }
-        Path path = new Path(fileSplit.getPath());
+        Path path = new Path(fileSplit.getLocalFile().getFile().getPath());
         Path tempOutputFile = null;
         Path finalOutputFile = null;
         checkIfCanWriteToHDFS(new FileSplit[] { fileSplit });
-        Object recordWriter  = null;
+        Object recordWriter = null;
         Object outputFormat = null;
         String taskAttempId = new TaskAttemptID().toString();
-        conf.set("mapred.task.id",taskAttempId);
+        conf.set("mapred.task.id", taskAttempId);
         outputPath = new Path(conf.get("mapred.output.dir"));
-        outputTempPath = new Path(outputPath,"_temporary");
-        if(outputPath != null && !fileSystem.exists(outputPath)) {
+        outputTempPath = new Path(outputPath, "_temporary");
+        if (outputPath != null && !fileSystem.exists(outputPath)) {
             fileSystem.mkdirs(outputTempPath);
         }
-        String suffix =  new String("part-r-00000");
+        String suffix = new String("part-r-00000");
         suffix = new String(suffix.substring(0, suffix.length() - ("" + index).length()));
         suffix = suffix + index;
-        tempOutputFile = new Path(outputTempPath,"_" + taskAttempId + "/" + suffix);
-        if (conf.getNumReduceTasks() == 0 ) {
+        tempOutputFile = new Path(outputTempPath, "_" + taskAttempId + "/" + suffix);
+        if (conf.getNumReduceTasks() == 0) {
             suffix.replace("-r-", "-m-");
         }
-        finalOutputFile = new Path(outputPath,suffix);
-        if(conf.getUseNewMapper()){
-            org.apache.hadoop.mapreduce.OutputFormat newOutputFormat = (org.apache.hadoop.mapreduce.OutputFormat)ReflectionUtils.newInstance((new JobContext(conf,null)).getOutputFormatClass(),conf);
-                recordWriter = newOutputFormat.getRecordWriter(new TaskAttemptContext(conf, new TaskAttemptID()));
-        }else {
-           recordWriter = conf.getOutputFormat().getRecordWriter(fileSystem, conf,suffix, new Progressable() {
-           @Override
-           public void progress() {}
-           });
+        finalOutputFile = new Path(outputPath, suffix);
+        if (conf.getUseNewMapper()) {
+            org.apache.hadoop.mapreduce.OutputFormat newOutputFormat = (org.apache.hadoop.mapreduce.OutputFormat) ReflectionUtils
+                    .newInstance((new JobContext(conf, null)).getOutputFormatClass(), conf);
+            recordWriter = newOutputFormat.getRecordWriter(new TaskAttemptContext(conf, new TaskAttemptID()));
+        } else {
+            recordWriter = conf.getOutputFormat().getRecordWriter(fileSystem, conf, suffix, new Progressable() {
+                @Override
+                public void progress() {
+                }
+            });
         }
-        
-        
+
         return new HadoopFileWriter(recordWriter, tempOutputFile, finalOutputFile, conf);
     }
-    
 
     Path outputPath;
     Path outputTempPath;
-   
+
     protected Reporter createReporter() {
-    return new Reporter() {
-        @Override
-        public Counter getCounter(Enum<?> name) {
-            return null;
-        }
+        return new Reporter() {
+            @Override
+            public Counter getCounter(Enum<?> name) {
+                return null;
+            }
 
-        @Override
-        public Counter getCounter(String group, String name) {
-            return null;
-        }
+            @Override
+            public Counter getCounter(String group, String name) {
+                return null;
+            }
 
-        @Override
-        public InputSplit getInputSplit() throws UnsupportedOperationException {
-            return null;
-        }
+            @Override
+            public InputSplit getInputSplit() throws UnsupportedOperationException {
+                return null;
+            }
 
-        @Override
-        public void incrCounter(Enum<?> key, long amount) {
+            @Override
+            public void incrCounter(Enum<?> key, long amount) {
 
-        }
+            }
 
-        @Override
-        public void incrCounter(String group, String counter, long amount) {
+            @Override
+            public void incrCounter(String group, String counter, long amount) {
 
-        }
+            }
 
-        @Override
-        public void progress() {
+            @Override
+            public void progress() {
 
-        }
+            }
 
-        @Override
-        public void setStatus(String status) {
+            @Override
+            public void setStatus(String status) {
 
-        }
-    };
-}
+            }
+        };
+    }
 
     private boolean checkIfCanWriteToHDFS(FileSplit[] fileSplits) throws Exception {
         JobConf conf = DatatypeHelper.map2JobConf((HashMap) jobConfMap);
         try {
             FileSystem fileSystem = FileSystem.get(conf);
             for (FileSplit fileSplit : fileSplits) {
-                Path path = new Path(fileSplit.getPath());
+                Path path = new Path(fileSplit.getLocalFile().getFile().getPath());
                 if (fileSystem.exists(path)) {
                     throw new Exception(" Output path :  already exists : " + path);
                 }
@@ -219,8 +218,8 @@
     private static FileSplit[] getOutputSplits(JobConf conf, int noOfMappers) throws ClassNotFoundException {
         int numOutputters = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : noOfMappers;
         Object outputFormat = null;
-        if(conf.getUseNewMapper()) {
-            outputFormat = ReflectionUtils.newInstance(new JobContext(conf,null).getOutputFormatClass(), conf);
+        if (conf.getUseNewMapper()) {
+            outputFormat = ReflectionUtils.newInstance(new JobContext(conf, null).getOutputFormatClass(), conf);
         } else {
             outputFormat = conf.getOutputFormat();
         }
@@ -228,7 +227,7 @@
             FileSplit[] outputFileSplits = new FileSplit[numOutputters];
             for (int i = 0; i < numOutputters; i++) {
                 String outputPath = "/tmp/" + System.currentTimeMillis() + i;
-                outputFileSplits[i] = new FileSplit("localhost", new File(outputPath));
+                outputFileSplits[i] = new FileSplit("localhost", new FileReference(new File(outputPath)));
             }
             return outputFileSplits;
         } else {
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorFactory.java
index ded1568..a679a05 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorFactory.java
@@ -17,7 +17,7 @@
 import java.io.DataOutput;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -35,7 +35,7 @@
     }
 
     @Override
-    public IAccumulatingAggregator createAggregator(IHyracksContext ctx, RecordDescriptor inRecordDesc,
+    public IAccumulatingAggregator createAggregator(IHyracksStageletContext ctx, RecordDescriptor inRecordDesc,
             final RecordDescriptor outRecordDescriptor) {
         final IFieldValueResultingAggregator aggregators[] = new IFieldValueResultingAggregator[aFactories.length];
         for (int i = 0; i < aFactories.length; ++i) {
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/HashDataWriter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/HashDataWriter.java
index 5e934c8..44dd84c 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/HashDataWriter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/HashDataWriter.java
@@ -18,7 +18,7 @@
 import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -33,21 +33,22 @@
     private final FrameTupleAccessor tupleAccessor;
     private final ITuplePartitionComputer tpc;
 
-    public HashDataWriter(IHyracksContext ctx, int consumerPartitionCount, IEndpointDataWriterFactory edwFactory,
-            RecordDescriptor recordDescriptor, ITuplePartitionComputer tpc) throws HyracksDataException {
+    public HashDataWriter(IHyracksStageletContext ctx, int consumerPartitionCount,
+            IEndpointDataWriterFactory edwFactory, RecordDescriptor recordDescriptor, ITuplePartitionComputer tpc)
+            throws HyracksDataException {
         this.consumerPartitionCount = consumerPartitionCount;
         epWriters = new IFrameWriter[consumerPartitionCount];
         appenders = new FrameTupleAppender[consumerPartitionCount];
         for (int i = 0; i < consumerPartitionCount; ++i) {
             try {
                 epWriters[i] = edwFactory.createFrameWriter(i);
-                appenders[i] = new FrameTupleAppender(ctx);
-                appenders[i].reset(ctx.getResourceManager().allocateFrame(), true);
+                appenders[i] = new FrameTupleAppender(ctx.getFrameSize());
+                appenders[i].reset(ctx.allocateFrame(), true);
             } catch (IOException e) {
                 throw new HyracksDataException(e);
             }
         }
-        tupleAccessor = new FrameTupleAccessor(ctx, recordDescriptor);
+        tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
         this.tpc = tpc;
     }
 
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNHashPartitioningConnectorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNHashPartitioningConnectorDescriptor.java
index c38a724..b814867 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNHashPartitioningConnectorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNHashPartitioningConnectorDescriptor.java
@@ -17,7 +17,7 @@
 import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -36,7 +36,7 @@
     }
 
     @Override
-    public IFrameWriter createSendSideWriter(IHyracksContext ctx, RecordDescriptor recordDesc,
+    public IFrameWriter createSendSideWriter(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
             IEndpointDataWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
             throws HyracksDataException {
         final HashDataWriter hashWriter = new HashDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc,
@@ -45,7 +45,7 @@
     }
 
     @Override
-    public IFrameReader createReceiveSideReader(IHyracksContext ctx, RecordDescriptor recordDesc,
+    public IFrameReader createReceiveSideReader(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
             IConnectionDemultiplexer demux, int index, int nProducerPartitions, int nConsumerPartitions)
             throws HyracksDataException {
         return new NonDeterministicFrameReader(ctx, demux);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNHashPartitioningMergingConnectorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNHashPartitioningMergingConnectorDescriptor.java
index 63075c8..36f8d5b 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNHashPartitioningMergingConnectorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNHashPartitioningMergingConnectorDescriptor.java
@@ -17,7 +17,7 @@
 import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -44,7 +44,7 @@
     }
 
     @Override
-    public IFrameWriter createSendSideWriter(IHyracksContext ctx, RecordDescriptor recordDesc,
+    public IFrameWriter createSendSideWriter(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
             IEndpointDataWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
             throws HyracksDataException {
         final HashDataWriter hashWriter = new HashDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc,
@@ -53,7 +53,7 @@
     }
 
     @Override
-    public IFrameReader createReceiveSideReader(IHyracksContext ctx, RecordDescriptor recordDesc,
+    public IFrameReader createReceiveSideReader(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
             IConnectionDemultiplexer demux, int index, int nProducerPartitions, int nConsumerPartitions)
             throws HyracksDataException {
         IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java
index 5346334..e108396 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java
@@ -20,7 +20,7 @@
 import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -38,11 +38,11 @@
         private final FrameTupleAppender[] appenders;
         private final FrameTupleAccessor tupleAccessor;
 
-        public RangeDataWriter(IHyracksContext ctx, int consumerPartitionCount, IFrameWriter[] epWriters,
+        public RangeDataWriter(IHyracksStageletContext ctx, int consumerPartitionCount, IFrameWriter[] epWriters,
                 FrameTupleAppender[] appenders, RecordDescriptor recordDescriptor) {
             this.epWriters = epWriters;
             this.appenders = appenders;
-            tupleAccessor = new FrameTupleAccessor(ctx, recordDescriptor);
+            tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
         }
 
         @Override
@@ -116,7 +116,7 @@
     }
 
     @Override
-    public IFrameWriter createSendSideWriter(IHyracksContext ctx, RecordDescriptor recordDesc,
+    public IFrameWriter createSendSideWriter(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
             IEndpointDataWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
             throws HyracksDataException {
         final IFrameWriter[] epWriters = new IFrameWriter[nConsumerPartitions];
@@ -124,8 +124,8 @@
         for (int i = 0; i < nConsumerPartitions; ++i) {
             try {
                 epWriters[i] = edwFactory.createFrameWriter(i);
-                appenders[i] = new FrameTupleAppender(ctx);
-                appenders[i].reset(ctx.getResourceManager().allocateFrame(), true);
+                appenders[i] = new FrameTupleAppender(ctx.getFrameSize());
+                appenders[i].reset(ctx.allocateFrame(), true);
             } catch (IOException e) {
                 throw new HyracksDataException(e);
             }
@@ -136,7 +136,7 @@
     }
 
     @Override
-    public IFrameReader createReceiveSideReader(IHyracksContext ctx, RecordDescriptor recordDesc,
+    public IFrameReader createReceiveSideReader(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
             IConnectionDemultiplexer demux, int index, int nProducerPartitions, int nConsumerPartitions)
             throws HyracksDataException {
         return new NonDeterministicFrameReader(ctx, demux);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNReplicatingConnectorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNReplicatingConnectorDescriptor.java
index 1ee7fda..37f68b7 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNReplicatingConnectorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNReplicatingConnectorDescriptor.java
@@ -19,7 +19,7 @@
 import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -35,7 +35,7 @@
     private static final long serialVersionUID = 1L;
 
     @Override
-    public IFrameWriter createSendSideWriter(IHyracksContext ctx, RecordDescriptor recordDesc,
+    public IFrameWriter createSendSideWriter(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
             IEndpointDataWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
             throws HyracksDataException {
         final IFrameWriter[] epWriters = new IFrameWriter[nConsumerPartitions];
@@ -78,7 +78,7 @@
     }
 
     @Override
-    public IFrameReader createReceiveSideReader(IHyracksContext ctx, RecordDescriptor recordDesc,
+    public IFrameReader createReceiveSideReader(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
             IConnectionDemultiplexer demux, int index, int nProducerPartitions, int nConsumerPartitions)
             throws HyracksDataException {
         return new NonDeterministicFrameReader(ctx, demux);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java
index e4595fb..e1154e4 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java
@@ -17,7 +17,7 @@
 import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -33,14 +33,14 @@
     }
 
     @Override
-    public IFrameWriter createSendSideWriter(IHyracksContext ctx, RecordDescriptor recordDesc,
+    public IFrameWriter createSendSideWriter(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
             IEndpointDataWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
             throws HyracksDataException {
         return edwFactory.createFrameWriter(index);
     }
 
     @Override
-    public IFrameReader createReceiveSideReader(IHyracksContext ctx, RecordDescriptor recordDesc,
+    public IFrameReader createReceiveSideReader(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
             IConnectionDemultiplexer demux, int index, int nProducerPartitions, int nConsumerPartitions)
             throws HyracksDataException {
         return new NonDeterministicFrameReader(ctx, demux);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractDeserializedFileScanOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractDeserializedFileScanOperatorDescriptor.java
index cea61f1..b2265dd 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractDeserializedFileScanOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractDeserializedFileScanOperatorDescriptor.java
@@ -16,7 +16,7 @@
 
 import java.io.File;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -65,7 +65,7 @@
             FileSplit split = splits[index];
             RecordDescriptor desc = recordDescriptors[0];
             try {
-                IRecordReader reader = createRecordReader(split.getLocalFile(), desc);
+                IRecordReader reader = createRecordReader(split.getLocalFile().getFile(), desc);
                 if (desc == null) {
                     desc = recordDescriptors[0];
                 }
@@ -100,7 +100,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new DeserializedOperatorNodePushable(ctx, new DeserializedFileScanOperator(partition), null);
     }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileWriteOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileWriteOperatorDescriptor.java
index 9ee7411..8e4199b 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileWriteOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileWriteOperatorDescriptor.java
@@ -14,9 +14,7 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.file;
 
-import java.io.File;
-
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -87,7 +85,7 @@
     protected abstract IRecordWriter createRecordWriter(FileSplit fileSplit, int index) throws Exception;
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new DeserializedOperatorNodePushable(ctx, new FileWriteOperator(partition),
                 recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java
index 716f587..954d64a 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java
@@ -23,7 +23,7 @@
 import java.util.Arrays;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
@@ -42,7 +42,7 @@
     }
 
     @Override
-    public ITupleParser createTupleParser(final IHyracksContext ctx) {
+    public ITupleParser createTupleParser(final IHyracksStageletContext ctx) {
         return new ITupleParser() {
             @Override
             public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
@@ -51,8 +51,8 @@
                     for (int i = 0; i < valueParserFactories.length; ++i) {
                         valueParsers[i] = valueParserFactories[i].createValueParser();
                     }
-                    ByteBuffer frame = ctx.getResourceManager().allocateFrame();
-                    FrameTupleAppender appender = new FrameTupleAppender(ctx);
+                    ByteBuffer frame = ctx.allocateFrame();
+                    FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
                     appender.reset(frame, true);
                     ArrayTupleBuilder tb = new ArrayTupleBuilder(valueParsers.length);
                     DataOutput dos = tb.getDataOutput();
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java
index 8b63690..7cbb2ed 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java
@@ -19,7 +19,7 @@
 import java.io.FileNotFoundException;
 import java.io.InputStream;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -45,14 +45,14 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         final FileSplit split = fileSplitProvider.getFileSplits()[partition];
         final ITupleParser tp = tupleParserFactory.createTupleParser(ctx);
         return new AbstractUnaryOutputSourceOperatorNodePushable() {
             @Override
             public void initialize() throws HyracksDataException {
-                File f = split.getLocalFile();
+                File f = split.getLocalFile().getFile();
                 writer.open();
                 try {
                     InputStream in;
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileSplit.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileSplit.java
index 254e8b0..b74d97b 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileSplit.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileSplit.java
@@ -17,36 +17,30 @@
 import java.io.File;
 import java.io.Serializable;
 
+import edu.uci.ics.hyracks.api.io.FileReference;
+
 public class FileSplit implements Serializable {
     private static final long serialVersionUID = 1L;
 
     private final String nodeName;
 
-    private final File file;
-    
-    private final String path;
+    private final FileReference file;
 
-    public FileSplit(String nodeName, File file) {
+    public FileSplit(String nodeName, FileReference file) {
         this.nodeName = nodeName;
         this.file = file;
-        this.path = file.getAbsolutePath();
     }
 
     public FileSplit(String nodeName, String path) {
-    	this.nodeName = nodeName;
-    	this.path = path;
-    	this.file = new File(path);
+        this.nodeName = nodeName;
+        this.file = new FileReference(new File(path));
     }
-    
+
     public String getNodeName() {
         return nodeName;
     }
 
-    public File getLocalFile() {
+    public FileReference getLocalFile() {
         return file;
     }
-    
-    public String getPath(){
-    	return path;
-    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FrameFileWriterOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FrameFileWriterOperatorDescriptor.java
index f36731a..1628984 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FrameFileWriterOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FrameFileWriterOperatorDescriptor.java
@@ -20,7 +20,7 @@
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -40,7 +40,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
         final FileSplit[] splits = fileSplitProvider.getFileSplits();
         return new AbstractUnaryInputSinkOperatorNodePushable() {
@@ -49,7 +49,7 @@
             @Override
             public void open() throws HyracksDataException {
                 try {
-                    out = new FileOutputStream(splits[partition].getLocalFile());
+                    out = new FileOutputStream(splits[partition].getLocalFile().getFile());
                 } catch (FileNotFoundException e) {
                     throw new HyracksDataException(e);
                 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ITupleParserFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ITupleParserFactory.java
index 708742b..a11dae0 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ITupleParserFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ITupleParserFactory.java
@@ -16,8 +16,8 @@
 
 import java.io.Serializable;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 
 public interface ITupleParserFactory extends Serializable {
-    public ITupleParser createTupleParser(IHyracksContext ctx);
+    public ITupleParser createTupleParser(IHyracksStageletContext ctx);
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/LineFileWriteOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/LineFileWriteOperatorDescriptor.java
index 6a11fdf..0760fa5 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/LineFileWriteOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/LineFileWriteOperatorDescriptor.java
@@ -14,58 +14,47 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.file;
 
-import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileOutputStream;
-import java.io.IOException;
 import java.io.OutputStream;
-import java.io.OutputStreamWriter;
 
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 
 public class LineFileWriteOperatorDescriptor extends AbstractFileWriteOperatorDescriptor {
-   
-	private static class LineWriterImpl extends RecordWriter {
-		
-		File file;
-		
-        LineWriterImpl(File file,int []columns, char separator) throws Exception {
-        	super(columns,separator, new Object[]{file});
-        	this.file=file;    
+    private static final long serialVersionUID = 1L;
+
+    private static class LineWriterImpl extends RecordWriter {
+        LineWriterImpl(File file, int[] columns, char separator) throws Exception {
+            super(columns, separator, new Object[] { file });
         }
 
-	    private static final long serialVersionUID = 1L;
-	
-		@Override
-		public OutputStream createOutputStream(Object[] args) throws Exception {
-			return new FileOutputStream((File)args[0]);
-		}
-	}
-    
-    private int[]columns ;
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public OutputStream createOutputStream(Object[] args) throws Exception {
+            return new FileOutputStream((File) args[0]);
+        }
+    }
+
+    private int[] columns;
     private char separator;
-    
-    
-	public LineFileWriteOperatorDescriptor(JobSpecification spec,
-            FileSplit[] splits) {
+
+    public LineFileWriteOperatorDescriptor(JobSpecification spec, FileSplit[] splits) {
         this(spec, splits, null, RecordWriter.COMMA);
     }
 
-    public LineFileWriteOperatorDescriptor(JobSpecification spec,
-            FileSplit[] splits, int[] columns) {
+    public LineFileWriteOperatorDescriptor(JobSpecification spec, FileSplit[] splits, int[] columns) {
         this(spec, splits, columns, RecordWriter.COMMA);
     }
 
-    public LineFileWriteOperatorDescriptor(JobSpecification spec,
-            FileSplit[] splits, int[] columns, char separator) {
+    public LineFileWriteOperatorDescriptor(JobSpecification spec, FileSplit[] splits, int[] columns, char separator) {
         super(spec, splits);
         this.columns = columns;
         this.separator = separator;
     }
-	
-	
+
     @Override
-    protected IRecordWriter createRecordWriter(FileSplit fileSplit,int index) throws Exception {
-        return new LineWriterImpl(fileSplit.getLocalFile(),columns,separator);
+    protected IRecordWriter createRecordWriter(FileSplit fileSplit, int index) throws Exception {
+        return new LineWriterImpl(fileSplit.getLocalFile().getFile(), columns, separator);
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/GroupingHashTable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/GroupingHashTable.java
index f25217e..07a5b81 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/GroupingHashTable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/GroupingHashTable.java
@@ -20,7 +20,7 @@
 import java.util.List;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -60,7 +60,7 @@
     }
 
     private static final int INIT_ACCUMULATORS_SIZE = 8;
-    private final IHyracksContext ctx;
+    private final IHyracksStageletContext ctx;
     private final FrameTupleAppender appender;
     private final List<ByteBuffer> buffers;
     private final Link[] table;
@@ -79,11 +79,11 @@
 
     private final FrameTupleAccessor storedKeysAccessor;
 
-    GroupingHashTable(IHyracksContext ctx, int[] fields, IBinaryComparatorFactory[] comparatorFactories,
+    GroupingHashTable(IHyracksStageletContext ctx, int[] fields, IBinaryComparatorFactory[] comparatorFactories,
             ITuplePartitionComputerFactory tpcf, IAccumulatingAggregatorFactory aggregatorFactory,
             RecordDescriptor inRecordDescriptor, RecordDescriptor outRecordDescriptor, int tableSize) {
         this.ctx = ctx;
-        appender = new FrameTupleAppender(ctx);
+        appender = new FrameTupleAppender(ctx.getFrameSize());
         buffers = new ArrayList<ByteBuffer>();
         table = new Link[tableSize];
         accumulators = new IAccumulatingAggregator[INIT_ACCUMULATORS_SIZE];
@@ -105,13 +105,13 @@
         this.inRecordDescriptor = inRecordDescriptor;
         this.outRecordDescriptor = outRecordDescriptor;
         RecordDescriptor storedKeysRecordDescriptor = new RecordDescriptor(storedKeySerDeser);
-        storedKeysAccessor = new FrameTupleAccessor(ctx, storedKeysRecordDescriptor);
+        storedKeysAccessor = new FrameTupleAccessor(ctx.getFrameSize(), storedKeysRecordDescriptor);
         lastBIndex = -1;
         addNewBuffer();
     }
 
     private void addNewBuffer() {
-        ByteBuffer buffer = ctx.getResourceManager().allocateFrame();
+        ByteBuffer buffer = ctx.allocateFrame();
         buffer.position(0);
         buffer.limit(buffer.capacity());
         buffers.add(buffer);
@@ -168,7 +168,7 @@
     }
 
     void write(IFrameWriter writer) throws HyracksDataException {
-        ByteBuffer buffer = ctx.getResourceManager().allocateFrame();
+        ByteBuffer buffer = ctx.allocateFrame();
         appender.reset(buffer, true);
         for (int i = 0; i < table.length; ++i) {
             Link link = table[i];
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashGroupOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashGroupOperatorDescriptor.java
index 09c1c6a..50d467c 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashGroupOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashGroupOperatorDescriptor.java
@@ -16,7 +16,7 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
@@ -73,9 +73,10 @@
         private static final long serialVersionUID = 1L;
 
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
-                final IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
-            final FrameTupleAccessor accessor = new FrameTupleAccessor(ctx,
+        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
+                final IOperatorEnvironment env, final IRecordDescriptorProvider recordDescProvider, int partition,
+                int nPartitions) {
+            final FrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(),
                     recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
             return new AbstractUnaryInputSinkOperatorNodePushable() {
                 private GroupingHashTable table;
@@ -117,7 +118,7 @@
         private static final long serialVersionUID = 1L;
 
         @Override
-        public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, final IOperatorEnvironment env,
+        public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, final IOperatorEnvironment env,
                 IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
             return new AbstractUnaryOutputSourceOperatorNodePushable() {
                 @Override
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregatorFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregatorFactory.java
index 97d8736..0d269e7 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregatorFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregatorFactory.java
@@ -16,11 +16,11 @@
 
 import java.io.Serializable;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 public interface IAccumulatingAggregatorFactory extends Serializable {
-    IAccumulatingAggregator createAggregator(IHyracksContext ctx, RecordDescriptor inRecordDesc,
+    IAccumulatingAggregator createAggregator(IHyracksStageletContext ctx, RecordDescriptor inRecordDesc,
             RecordDescriptor outRecordDescriptor) throws HyracksDataException;
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupOperatorDescriptor.java
index 3337385..ea86d19 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupOperatorDescriptor.java
@@ -16,7 +16,7 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -48,7 +48,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
         final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
         for (int i = 0; i < comparatorFactories.length; ++i) {
@@ -57,11 +57,11 @@
         final RecordDescriptor inRecordDesc = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
         final IAccumulatingAggregator aggregator = aggregatorFactory.createAggregator(ctx, inRecordDesc,
                 recordDescriptors[0]);
-        final ByteBuffer copyFrame = ctx.getResourceManager().allocateFrame();
-        final FrameTupleAccessor copyFrameAccessor = new FrameTupleAccessor(ctx, inRecordDesc);
+        final ByteBuffer copyFrame = ctx.allocateFrame();
+        final FrameTupleAccessor copyFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDesc);
         copyFrameAccessor.reset(copyFrame);
-        ByteBuffer outFrame = ctx.getResourceManager().allocateFrame();
-        final FrameTupleAppender appender = new FrameTupleAppender(ctx);
+        ByteBuffer outFrame = ctx.allocateFrame();
+        final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
         appender.reset(outFrame, true);
         return new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
             private PreclusteredGroupWriter pgw;
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupWriter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupWriter.java
index 93c8b3f..8817113 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupWriter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupWriter.java
@@ -17,7 +17,7 @@
 import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -37,18 +37,18 @@
     private final FrameTupleAppender appender;
     private boolean first;
 
-    public PreclusteredGroupWriter(IHyracksContext ctx, int[] groupFields, IBinaryComparator[] comparators,
+    public PreclusteredGroupWriter(IHyracksStageletContext ctx, int[] groupFields, IBinaryComparator[] comparators,
             IAccumulatingAggregator aggregator, RecordDescriptor inRecordDesc, IFrameWriter writer) {
         this.groupFields = groupFields;
         this.comparators = comparators;
         this.aggregator = aggregator;
         this.writer = writer;
-        copyFrame = ctx.getResourceManager().allocateFrame();
-        inFrameAccessor = new FrameTupleAccessor(ctx, inRecordDesc);
-        copyFrameAccessor = new FrameTupleAccessor(ctx, inRecordDesc);
+        copyFrame = ctx.allocateFrame();
+        inFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDesc);
+        copyFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDesc);
         copyFrameAccessor.reset(copyFrame);
-        outFrame = ctx.getResourceManager().allocateFrame();
-        appender = new FrameTupleAppender(ctx);
+        outFrame = ctx.allocateFrame();
+        appender = new FrameTupleAppender(ctx.getFrameSize());
         appender.reset(outFrame, true);
     }
 
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorDescriptor.java
index 221c22f..bff101e 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorDescriptor.java
@@ -14,13 +14,9 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.join;
 
-import java.io.File;
-import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
@@ -31,6 +27,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
@@ -39,6 +36,8 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.partition.RepartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileReader;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractActivityNode;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
@@ -47,7 +46,7 @@
 public class GraceHashJoinOperatorDescriptor extends AbstractOperatorDescriptor {
     private static final String SMALLRELATION = "RelR";
     private static final String LARGERELATION = "RelS";
-    private static final String NUM_PARTITION = "NUMBER_PARTITIONS";
+
     private static final long serialVersionUID = 1L;
     private final int[] keys0;
     private final int[] keys1;
@@ -58,10 +57,6 @@
     private final IBinaryHashFunctionFactory[] hashFunctionFactories;
     private final IBinaryComparatorFactory[] comparatorFactories;
 
-    private int numReadI1 = 0;
-    private int numWrite = 0;
-    private int numReadI2 = 0;
-
     public GraceHashJoinOperatorDescriptor(JobSpecification spec, int memsize, int inputsize0, int recordsPerFrame,
             double factor, int[] keys0, int[] keys1, IBinaryHashFunctionFactory[] hashFunctionFactories,
             IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor) {
@@ -102,58 +97,66 @@
 
     private class HashPartitionActivityNode extends AbstractActivityNode {
         private static final long serialVersionUID = 1L;
-        private String relationName;
+        private String partitionsKey;
         private int operatorInputIndex;
         private int keys[];
 
-        public HashPartitionActivityNode(String relationName, int keys[], int operatorInputIndex) {
-            this.relationName = relationName;
+        public HashPartitionActivityNode(String partitionsKey, int keys[], int operatorInputIndex) {
+            this.partitionsKey = partitionsKey;
             this.keys = keys;
             this.operatorInputIndex = operatorInputIndex;
         }
 
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
-                final IRecordDescriptorProvider recordDescProvider, int partition, final int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
+                final IOperatorEnvironment env, final IRecordDescriptorProvider recordDescProvider, int partition,
+                final int nPartitions) {
             final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
             for (int i = 0; i < comparatorFactories.length; ++i) {
                 comparators[i] = comparatorFactories[i].createBinaryComparator();
             }
             IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
-                private final FrameTupleAccessor accessor0 = new FrameTupleAccessor(ctx,
+                private final FrameTupleAccessor accessor0 = new FrameTupleAccessor(ctx.getFrameSize(),
                         recordDescProvider.getInputRecordDescriptor(getOperatorId(), operatorInputIndex));
 
-                ITuplePartitionComputer hpc = new FieldHashPartitionComputerFactory(keys, hashFunctionFactories)
-                        .createPartitioner();
+                private final ITuplePartitionComputer hpc = new FieldHashPartitionComputerFactory(keys,
+                        hashFunctionFactories).createPartitioner();
 
-                private final FrameTupleAppender appender = new FrameTupleAppender(ctx);
+                private final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
                 private ByteBuffer[] outbufs;
-                private File[] files;
-                private FileChannel[] channels;
+                private RunFileWriter[] fWriters;
                 private final int numPartitions = (int) Math.ceil(Math.sqrt(inputsize0 * factor / nPartitions));
 
                 @Override
                 public void close() throws HyracksDataException {
                     for (int i = 0; i < numPartitions; i++) {
-                        try {
-                            ByteBuffer head = outbufs[i];
-                            accessor0.reset(head);
-                            if (accessor0.getTupleCount() > 0) {
-                                FileChannel wChannel = channels[i];
-                                if (wChannel == null) {
-                                    wChannel = new RandomAccessFile(files[i], "rw").getChannel();
-                                    channels[i] = wChannel;
-                                }
-                                wChannel.write(head);
-                                numWrite++;
-                            }
-                        } catch (IOException e) {
-                            throw new HyracksDataException("error generating partition " + files[i].getName());
+                        ByteBuffer head = outbufs[i];
+                        accessor0.reset(head);
+                        if (accessor0.getTupleCount() > 0) {
+                            write(i, head);
                         }
+                        closeWriter(i);
                     }
 
-                    env.set(relationName, channels);
-                    env.set(NUM_PARTITION, numPartitions);
+                    env.set(partitionsKey, fWriters);
+                }
+
+                private void closeWriter(int i) throws HyracksDataException {
+                    RunFileWriter writer = fWriters[i];
+                    if (writer != null) {
+                        writer.close();
+                    }
+                }
+
+                private void write(int i, ByteBuffer head) throws HyracksDataException {
+                    RunFileWriter writer = fWriters[i];
+                    if (writer == null) {
+                        FileReference file = ctx.getJobletContext().createWorkspaceFile(partitionsKey);
+                        writer = new RunFileWriter(file, ctx.getIOManager());
+                        writer.open();
+                        fWriters[i] = writer;
+                    }
+                    writer.nextFrame(head);
                 }
 
                 @Override
@@ -163,31 +166,17 @@
                     for (int i = 0; i < tCount; ++i) {
 
                         int entry = hpc.partition(accessor0, i, numPartitions);
-                        boolean newBuffer = false;
                         ByteBuffer outbuf = outbufs[entry];
+                        appender.reset(outbuf, true);
                         while (true) {
-                            appender.reset(outbuf, newBuffer);
                             if (appender.append(accessor0, i)) {
                                 break;
                             } else {
                                 // buffer is full, ie. we cannot fit the tuple
                                 // into the buffer -- write it to disk
-                                try {
-
-                                    FileChannel wChannel = channels[entry];
-                                    if (wChannel == null) {
-                                        wChannel = new RandomAccessFile(files[entry], "rw").getChannel();
-                                        channels[entry] = wChannel;
-                                    }
-
-                                    wChannel.write(outbuf);
-                                    numWrite++;
-                                    outbuf.clear();
-                                    newBuffer = true;
-                                } catch (IOException e) {
-                                    throw new HyracksDataException("error generating partition "
-                                            + files[entry].getName());
-                                }
+                                write(entry, outbuf);
+                                outbuf.clear();
+                                appender.reset(outbuf, true);
                             }
                         }
                     }
@@ -196,16 +185,9 @@
                 @Override
                 public void open() throws HyracksDataException {
                     outbufs = new ByteBuffer[numPartitions];
-                    files = new File[numPartitions];
-                    channels = new FileChannel[numPartitions];
+                    fWriters = new RunFileWriter[numPartitions];
                     for (int i = 0; i < numPartitions; i++) {
-                        try {
-                            files[i] = ctx.getResourceManager().createFile(relationName, null);
-                            files[i].deleteOnExit();
-                            outbufs[i] = ctx.getResourceManager().allocateFrame();
-                        } catch (IOException e) {
-                            throw new HyracksDataException(e);
-                        }
+                        outbufs[i] = ctx.allocateFrame();
                     }
                 }
 
@@ -226,8 +208,9 @@
         private static final long serialVersionUID = 1L;
 
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
-                final IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
+                final IOperatorEnvironment env, final IRecordDescriptorProvider recordDescProvider, int partition,
+                final int nPartitions) {
             final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
             for (int i = 0; i < comparatorFactories.length; ++i) {
                 comparators[i] = comparatorFactories[i].createBinaryComparator();
@@ -238,16 +221,14 @@
             IOperatorNodePushable op = new AbstractUnaryOutputSourceOperatorNodePushable() {
                 private InMemoryHashJoin joiner;
 
-                private FileChannel[] channelsR;
-                private FileChannel[] channelsS;
-                private int numPartitions;
-                private int[] maxBufferRi;
+                private RunFileWriter[] rWriters;
+                private RunFileWriter[] sWriters;
+                private final int numPartitions = (int) Math.ceil(Math.sqrt(inputsize0 * factor / nPartitions));
 
                 @Override
                 public void initialize() throws HyracksDataException {
-                    channelsR = (FileChannel[]) env.get(SMALLRELATION);
-                    channelsS = (FileChannel[]) env.get(LARGERELATION);
-                    numPartitions = (Integer) env.get(NUM_PARTITION);
+                    rWriters = (RunFileWriter[]) env.get(SMALLRELATION);
+                    sWriters = (RunFileWriter[]) env.get(LARGERELATION);
 
                     ITuplePartitionComputer hpcRep0 = new RepartitionComputerFactory(numPartitions,
                             new FieldHashPartitionComputerFactory(keys0, hashFunctionFactories)).createPartitioner();
@@ -256,61 +237,39 @@
 
                     writer.open();// open for probe
 
-                    maxBufferRi = new int[numPartitions];
-
-                    ByteBuffer buffer = ctx.getResourceManager().allocateFrame();// input
+                    ByteBuffer buffer = ctx.allocateFrame();// input
                     // buffer
                     int tableSize = (int) (numPartitions * recordsPerFrame * factor);
                     for (int partitionid = 0; partitionid < numPartitions; partitionid++) {
-                        int counter = 0;
-                        int state = 0;
-                        try {
-                            FileChannel inChannelR = channelsR[partitionid];
-                            if (inChannelR != null) {
-                                inChannelR.position(0);
-                                while (state != -1) {
-
-                                    joiner = new InMemoryHashJoin(ctx, tableSize, new FrameTupleAccessor(ctx, rd0),
-                                            hpcRep0, new FrameTupleAccessor(ctx, rd1), hpcRep1,
-                                            new FrameTuplePairComparator(keys0, keys1, comparators));
-                                    // build
-
-                                    state = inChannelR.read(buffer);
-                                    while (state != -1) {
-
-                                        ByteBuffer copyBuffer = ctx.getResourceManager().allocateFrame();
-                                        FrameUtils.copy(buffer, copyBuffer);
-                                        joiner.build(copyBuffer);
-                                        numReadI1++;
-                                        counter++;
-                                        if (counter > maxBufferRi[partitionid])
-                                            maxBufferRi[partitionid] = counter;
-
-                                        buffer.clear();
-                                        state = inChannelR.read(buffer);
-                                    }
-
-                                    // probe
-
-                                    buffer.clear();
-
-                                    FileChannel inChannelS = channelsS[partitionid];
-                                    if (inChannelS != null) {
-                                        inChannelS.position(0);
-                                        while (inChannelS.read(buffer) != -1) {
-                                            joiner.join(buffer, writer);
-                                            numReadI2++;
-                                            buffer.clear();
-                                        }
-                                        inChannelS.close();
-                                        joiner.closeJoin(writer);
-                                    }
-                                }
-                                inChannelR.close();
-                            }
-                        } catch (IOException e) {
-                            throw new HyracksDataException(e);
+                        RunFileWriter rWriter = rWriters[partitionid];
+                        RunFileWriter sWriter = sWriters[partitionid];
+                        if (rWriter == null || sWriter == null) {
+                            continue;
                         }
+                        joiner = new InMemoryHashJoin(ctx, tableSize, new FrameTupleAccessor(ctx.getFrameSize(), rd0),
+                                hpcRep0, new FrameTupleAccessor(ctx.getFrameSize(), rd1), hpcRep1,
+                                new FrameTuplePairComparator(keys0, keys1, comparators));
+
+                        // build
+                        RunFileReader rReader = rWriter.createReader();
+                        rReader.open();
+                        while (rReader.nextFrame(buffer)) {
+                            ByteBuffer copyBuffer = ctx.allocateFrame();
+                            FrameUtils.copy(buffer, copyBuffer);
+                            joiner.build(copyBuffer);
+                            buffer.clear();
+                        }
+                        rReader.close();
+
+                        // probe
+                        RunFileReader sReader = sWriter.createReader();
+                        sReader.open();
+                        while (sReader.nextFrame(buffer)) {
+                            joiner.join(buffer, writer);
+                            buffer.clear();
+                        }
+                        sReader.close();
+                        joiner.closeJoin(writer);
                     }
                     writer.close();
                 }
@@ -319,7 +278,6 @@
                 public void deinitialize() throws HyracksDataException {
                     env.set(LARGERELATION, null);
                     env.set(SMALLRELATION, null);
-                    env.set(NUM_PARTITION, null);
                 }
             };
             return op;
@@ -330,4 +288,4 @@
             return GraceHashJoinOperatorDescriptor.this;
         }
     }
-}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
index a02156d..d7c1086 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
@@ -14,13 +14,9 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.join;
 
-import java.io.File;
-import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
@@ -32,6 +28,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
@@ -40,6 +37,8 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.partition.RepartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileReader;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractActivityNode;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
@@ -61,11 +60,6 @@
     private final IBinaryComparatorFactory[] comparatorFactories;
     private final int recordsPerFrame;
 
-    private int numReadI1 = 0;
-    private int numWriteI1 = 0;
-    private int numReadI2 = 0;
-    private int numWriteI2 = 0;
-
     /**
      * @param spec
      * @param memsize
@@ -124,8 +118,9 @@
         }
 
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
-                IRecordDescriptorProvider recordDescProvider, int partition, final int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
+                final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
+                final int nPartitions) {
             final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
             final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 1);
             final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
@@ -135,15 +130,14 @@
 
             IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
                 private InMemoryHashJoin joiner0;
-                private final FrameTupleAccessor accessor0 = new FrameTupleAccessor(ctx, rd0);
+                private final FrameTupleAccessor accessor0 = new FrameTupleAccessor(ctx.getFrameSize(), rd0);
                 ITuplePartitionComputer hpc0 = new FieldHashPartitionComputerFactory(keys0, hashFunctionFactories)
                         .createPartitioner();
-                private final FrameTupleAppender appender = new FrameTupleAppender(ctx);
-                private final FrameTupleAppender ftappender = new FrameTupleAppender(ctx);
+                private final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+                private final FrameTupleAppender ftappender = new FrameTupleAppender(ctx.getFrameSize());
                 private ByteBuffer[] bufferForPartitions;
-                private final ByteBuffer inBuffer = ctx.getResourceManager().allocateFrame();
-                private File[] files;
-                private FileChannel[] channels;
+                private final ByteBuffer inBuffer = ctx.allocateFrame();
+                private RunFileWriter[] fWriters;
                 private int memoryForHashtable;
                 private int B;
 
@@ -153,24 +147,15 @@
                         build(inBuffer);
 
                     for (int i = 0; i < B; i++) {
-                        try {
-                            ByteBuffer buf = bufferForPartitions[i];
-                            accessor0.reset(buf);
-                            if (accessor0.getTupleCount() > 0) {
-                                FileChannel wChannel = channels[i];
-                                if (wChannel == null) {
-                                    wChannel = new RandomAccessFile(files[i], "rw").getChannel();
-                                    channels[i] = wChannel;
-                                }
-                                wChannel.write(buf);
-                                numWriteI1++;
-                            }
-                        } catch (IOException e) {
-                            throw new HyracksDataException(e);
+                        ByteBuffer buf = bufferForPartitions[i];
+                        accessor0.reset(buf);
+                        if (accessor0.getTupleCount() > 0) {
+                            write(i, buf);
                         }
+                        closeWriter(i);
                     }
 
-                    env.set(relationName, channels);
+                    env.set(relationName, fWriters);
                     env.set(JOINER0, joiner0);
                     env.set(NUM_PARTITION, B);
                     env.set(MEM_HASHTABLE, memoryForHashtable);
@@ -193,19 +178,9 @@
                                     if (appender.append(accessor0, i)) {
                                         break;
                                     } else {
-                                        try {
-                                            FileChannel wChannel = channels[entry];
-                                            if (wChannel == null) {
-                                                wChannel = new RandomAccessFile(files[entry], "rw").getChannel();
-                                                channels[entry] = wChannel;
-                                            }
-                                            wChannel.write(bufBi);
-                                            numWriteI1++;
-                                            bufBi.clear();
-                                            newBuffer = true;
-                                        } catch (IOException e) {
-                                            throw new HyracksDataException(e);
-                                        }
+                                        write(entry, bufBi);
+                                        bufBi.clear();
+                                        newBuffer = true;
                                     }
                                 }
                             } else {
@@ -228,21 +203,9 @@
                                         if (appender.append(accessor0, i)) {
                                             break;
                                         } else {
-                                            try {
-                                                FileChannel wChannel;
-                                                if (channels[entry] == null) {
-                                                    wChannel = new RandomAccessFile(files[entry], "rw").getChannel();
-                                                    channels[entry] = wChannel;
-                                                } else {
-                                                    wChannel = channels[entry];
-                                                }
-                                                wChannel.write(bufBi);
-                                                numWriteI1++;
-                                                bufBi.clear();
-                                                newBuffer = true;
-                                            } catch (IOException e) {
-                                                throw new HyracksDataException(e);
-                                            }
+                                            write(entry, bufBi);
+                                            bufBi.clear();
+                                            newBuffer = true;
                                         }
                                     }
                                 }
@@ -256,7 +219,7 @@
                 }
 
                 private void build(ByteBuffer inBuffer) throws HyracksDataException {
-                    ByteBuffer copyBuffer = ctx.getResourceManager().allocateFrame();
+                    ByteBuffer copyBuffer = ctx.allocateFrame();
                     FrameUtils.copy(inBuffer, copyBuffer);
                     joiner0.build(copyBuffer);
                 }
@@ -290,19 +253,13 @@
                     ITuplePartitionComputer hpc1 = new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories)
                             .createPartitioner();
                     int tableSize = (int) (memoryForHashtable * recordsPerFrame * factor);
-                    joiner0 = new InMemoryHashJoin(ctx, tableSize, new FrameTupleAccessor(ctx, rd0), hpc0,
-                            new FrameTupleAccessor(ctx, rd1), hpc1, new FrameTuplePairComparator(keys0, keys1,
-                                    comparators));
-                    files = new File[B];
-                    channels = new FileChannel[B];
+                    joiner0 = new InMemoryHashJoin(ctx, tableSize, new FrameTupleAccessor(ctx.getFrameSize(), rd0),
+                            hpc0, new FrameTupleAccessor(ctx.getFrameSize(), rd1), hpc1, new FrameTuplePairComparator(
+                                    keys0, keys1, comparators));
                     bufferForPartitions = new ByteBuffer[B];
+                    fWriters = new RunFileWriter[B];
                     for (int i = 0; i < B; i++) {
-                        try {
-                            files[i] = ctx.getResourceManager().createFile(relationName, null);
-                            bufferForPartitions[i] = ctx.getResourceManager().allocateFrame();
-                        } catch (IOException e) {
-                            throw new HyracksDataException(e);
-                        }
+                        bufferForPartitions[i] = ctx.allocateFrame();
                     }
 
                     ftappender.reset(inBuffer, true);
@@ -311,6 +268,24 @@
                 @Override
                 public void flush() throws HyracksDataException {
                 }
+
+                private void closeWriter(int i) throws HyracksDataException {
+                    RunFileWriter writer = fWriters[i];
+                    if (writer != null) {
+                        writer.close();
+                    }
+                }
+
+                private void write(int i, ByteBuffer head) throws HyracksDataException {
+                    RunFileWriter writer = fWriters[i];
+                    if (writer == null) {
+                        FileReference file = ctx.getJobletContext().createWorkspaceFile(relationName);
+                        writer = new RunFileWriter(file, ctx.getIOManager());
+                        writer.open();
+                        fWriters[i] = writer;
+                    }
+                    writer.nextFrame(head);
+                }
             };
             return op;
         }
@@ -331,8 +306,9 @@
         }
 
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
-                IRecordDescriptorProvider recordDescProvider, int partition, final int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
+                final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
+                final int nPartitions) {
             final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
             final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 1);
             final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
@@ -342,20 +318,19 @@
 
             IOperatorNodePushable op = new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
                 private InMemoryHashJoin joiner0;
-                private final FrameTupleAccessor accessor1 = new FrameTupleAccessor(ctx, rd1);
+                private final FrameTupleAccessor accessor1 = new FrameTupleAccessor(ctx.getFrameSize(), rd1);
                 private ITuplePartitionComputerFactory hpcf0 = new FieldHashPartitionComputerFactory(keys0,
                         hashFunctionFactories);
                 private ITuplePartitionComputerFactory hpcf1 = new FieldHashPartitionComputerFactory(keys1,
                         hashFunctionFactories);
                 ITuplePartitionComputer hpc1 = hpcf1.createPartitioner();
 
-                private final FrameTupleAppender appender = new FrameTupleAppender(ctx);
-                private final FrameTupleAppender ftap = new FrameTupleAppender(ctx);
-                private final ByteBuffer inBuffer = ctx.getResourceManager().allocateFrame();
-                private final ByteBuffer outBuffer = ctx.getResourceManager().allocateFrame();
-                private FileChannel[] channelsR;
-                private FileChannel[] channelsS;
-                private File filesS[];
+                private final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+                private final FrameTupleAppender ftap = new FrameTupleAppender(ctx.getFrameSize());
+                private final ByteBuffer inBuffer = ctx.allocateFrame();
+                private final ByteBuffer outBuffer = ctx.allocateFrame();
+                private RunFileWriter[] rWriters;
+                private RunFileWriter[] sWriters;
                 private ByteBuffer[] bufferForPartitions;
                 private int B;
                 private int memoryForHashtable;
@@ -364,19 +339,13 @@
                 public void open() throws HyracksDataException {
                     joiner0 = (InMemoryHashJoin) env.get(JOINER0);
                     writer.open();
-                    channelsR = (FileChannel[]) env.get(SMALLRELATION);
+                    rWriters = (RunFileWriter[]) env.get(SMALLRELATION);
                     B = (Integer) env.get(NUM_PARTITION);
                     memoryForHashtable = (Integer) env.get(MEM_HASHTABLE);
-                    filesS = new File[B];
-                    channelsS = new FileChannel[B];
+                    sWriters = new RunFileWriter[B];
                     bufferForPartitions = new ByteBuffer[B];
                     for (int i = 0; i < B; i++) {
-                        try {
-                            filesS[i] = ctx.getResourceManager().createFile(largeRelation, null);
-                            bufferForPartitions[i] = ctx.getResourceManager().allocateFrame();
-                        } catch (IOException e) {
-                            throw new HyracksDataException(e);
-                        }
+                        bufferForPartitions[i] = ctx.allocateFrame();
                     }
                     appender.reset(outBuffer, true);
                     ftap.reset(inBuffer, true);
@@ -399,20 +368,9 @@
                                     if (appender.append(accessor1, i)) {
                                         break;
                                     } else {
-                                        try {
-                                            FileChannel wChannel = channelsS[entry];
-                                            if (wChannel == null) {
-                                                wChannel = new RandomAccessFile(filesS[entry], "rw").getChannel();
-                                                channelsS[entry] = wChannel;
-                                            }
-
-                                            wChannel.write(outbuf);
-                                            numWriteI2++;
-                                            outbuf.clear();
-                                            newBuffer = true;
-                                        } catch (IOException e) {
-                                            throw new HyracksDataException(e);
-                                        }
+                                        write(entry, outbuf);
+                                        outbuf.clear();
+                                        newBuffer = true;
                                     }
                                 }
                             } else {
@@ -435,19 +393,9 @@
                                         if (appender.append(accessor1, i)) {
                                             break;
                                         } else {
-                                            try {
-                                                FileChannel wChannel = channelsS[entry];
-                                                if (wChannel == null) {
-                                                    wChannel = new RandomAccessFile(filesS[entry], "rw").getChannel();
-                                                    channelsS[entry] = wChannel;
-                                                }
-                                                wChannel.write(outbuf);
-                                                numWriteI2++;
-                                                outbuf.clear();
-                                                newBuffer = true;
-                                            } catch (IOException e) {
-                                                throw new HyracksDataException(e);
-                                            }
+                                            write(entry, outbuf);
+                                            outbuf.clear();
+                                            newBuffer = true;
                                         }
                                     }
                                 }
@@ -466,21 +414,12 @@
                     ITuplePartitionComputer hpcRep1 = new RepartitionComputerFactory(B, hpcf1).createPartitioner();
                     if (memoryForHashtable != memsize - 2) {
                         for (int i = 0; i < B; i++) {
-                            try {
-                                ByteBuffer buf = bufferForPartitions[i];
-                                accessor1.reset(buf);
-                                if (accessor1.getTupleCount() > 0) {
-                                    FileChannel wChannel = channelsS[i];
-                                    if (wChannel == null) {
-                                        wChannel = new RandomAccessFile(filesS[i], "rw").getChannel();
-                                        channelsS[i] = wChannel;
-                                    }
-                                    wChannel.write(buf);
-                                    numWriteI2++;
-                                }
-                            } catch (IOException e) {
-                                throw new HyracksDataException(e);
+                            ByteBuffer buf = bufferForPartitions[i];
+                            accessor1.reset(buf);
+                            if (accessor1.getTupleCount() > 0) {
+                                write(i, buf);
                             }
+                            closeWriter(i);
                         }
 
                         inBuffer.clear();
@@ -491,45 +430,35 @@
                             tableSize = (int) (memsize * recordsPerFrame * factor);
                         }
                         for (int partitionid = 0; partitionid < B; partitionid++) {
-
-                            int state = 0;
-                            try {
-                                FileChannel inChannel = channelsR[partitionid];
-                                if (inChannel != null) {
-                                    inChannel.position(0);
-                                    InMemoryHashJoin joiner = new InMemoryHashJoin(ctx, tableSize,
-                                            new FrameTupleAccessor(ctx, rd0), hpcRep0,
-                                            new FrameTupleAccessor(ctx, rd1), hpcRep1, new FrameTuplePairComparator(
-                                                    keys0, keys1, comparators));
-                                    state = inChannel.read(inBuffer);
-                                    while (state != -1) {
-                                        numReadI1++;
-                                        ByteBuffer copyBuffer = ctx.getResourceManager().allocateFrame();
-                                        FrameUtils.copy(inBuffer, copyBuffer);
-                                        joiner.build(copyBuffer);
-                                        inBuffer.clear();
-                                        state = inChannel.read(inBuffer);
-                                    }
-                                    appender.reset(outBuffer, false);
-
-                                    inBuffer.clear();
-
-                                    FileChannel inChannelS = channelsS[partitionid];
-                                    if (inChannelS != null) {
-                                        inChannelS.position(0);
-                                        while (inChannelS.read(inBuffer) != -1) {
-                                            numReadI2++;
-                                            joiner.join(inBuffer, writer);
-                                            inBuffer.clear();
-                                        }
-                                        inChannelS.close();
-                                        joiner.closeJoin(writer);
-                                    }
-                                    inChannel.close();
-                                }
-                            } catch (IOException e) {
-                                throw new HyracksDataException(e);
+                            RunFileWriter rWriter = rWriters[partitionid];
+                            RunFileWriter sWriter = sWriters[partitionid];
+                            if (rWriter == null || sWriter == null) {
+                                continue;
                             }
+
+                            InMemoryHashJoin joiner = new InMemoryHashJoin(ctx, tableSize, new FrameTupleAccessor(
+                                    ctx.getFrameSize(), rd0), hpcRep0, new FrameTupleAccessor(ctx.getFrameSize(), rd1),
+                                    hpcRep1, new FrameTuplePairComparator(keys0, keys1, comparators));
+
+                            RunFileReader rReader = rWriter.createReader();
+                            rReader.open();
+                            while (rReader.nextFrame(inBuffer)) {
+                                ByteBuffer copyBuffer = ctx.allocateFrame();
+                                FrameUtils.copy(inBuffer, copyBuffer);
+                                joiner.build(copyBuffer);
+                                inBuffer.clear();
+                            }
+                            rReader.close();
+
+                            // probe
+                            RunFileReader sReader = sWriter.createReader();
+                            sReader.open();
+                            while (sReader.nextFrame(inBuffer)) {
+                                joiner.join(inBuffer, writer);
+                                inBuffer.clear();
+                            }
+                            sReader.close();
+                            joiner.closeJoin(writer);
                         }
                     }
                     writer.close();
@@ -544,6 +473,24 @@
                 public void flush() throws HyracksDataException {
                     writer.flush();
                 }
+
+                private void closeWriter(int i) throws HyracksDataException {
+                    RunFileWriter writer = sWriters[i];
+                    if (writer != null) {
+                        writer.close();
+                    }
+                }
+
+                private void write(int i, ByteBuffer head) throws HyracksDataException {
+                    RunFileWriter writer = sWriters[i];
+                    if (writer == null) {
+                        FileReference file = ctx.createWorkspaceFile(largeRelation);
+                        writer = new RunFileWriter(file, ctx.getIOManager());
+                        writer.open();
+                        sWriters[i] = writer;
+                    }
+                    writer.nextFrame(head);
+                }
             };
             return op;
         }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
index b2a9f39..4d98c6a 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
@@ -20,7 +20,7 @@
 import java.util.List;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
@@ -38,7 +38,7 @@
     private final FrameTuplePairComparator tpComparator;
     private final ByteBuffer outBuffer;
 
-    public InMemoryHashJoin(IHyracksContext ctx, int tableSize, FrameTupleAccessor accessor0,
+    public InMemoryHashJoin(IHyracksStageletContext ctx, int tableSize, FrameTupleAccessor accessor0,
             ITuplePartitionComputer tpc0, FrameTupleAccessor accessor1, ITuplePartitionComputer tpc1,
             FrameTuplePairComparator comparator) {
         table = new Link[tableSize];
@@ -47,9 +47,9 @@
         this.tpc0 = tpc0;
         this.accessor1 = accessor1;
         this.tpc1 = tpc1;
-        appender = new FrameTupleAppender(ctx);
+        appender = new FrameTupleAppender(ctx.getFrameSize());
         tpComparator = comparator;
-        outBuffer = ctx.getResourceManager().allocateFrame();
+        outBuffer = ctx.allocateFrame();
         appender.reset(outBuffer, true);
     }
 
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
index a987e68..74f0146 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
@@ -16,7 +16,7 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
@@ -79,8 +79,9 @@
         private static final long serialVersionUID = 1L;
 
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
-                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
+                final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
+                int nPartitions) {
             final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
             final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 1);
             final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
@@ -96,14 +97,14 @@
                             .createPartitioner();
                     ITuplePartitionComputer hpc1 = new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories)
                             .createPartitioner();
-                    joiner = new InMemoryHashJoin(ctx, tableSize, new FrameTupleAccessor(ctx, rd0), hpc0,
-                            new FrameTupleAccessor(ctx, rd1), hpc1, new FrameTuplePairComparator(keys0, keys1,
-                                    comparators));
+                    joiner = new InMemoryHashJoin(ctx, tableSize, new FrameTupleAccessor(ctx.getFrameSize(), rd0),
+                            hpc0, new FrameTupleAccessor(ctx.getFrameSize(), rd1), hpc1, new FrameTuplePairComparator(
+                                    keys0, keys1, comparators));
                 }
 
                 @Override
                 public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-                    ByteBuffer copyBuffer = ctx.getResourceManager().allocateFrame();
+                    ByteBuffer copyBuffer = ctx.allocateFrame();
                     FrameUtils.copy(buffer, copyBuffer);
                     joiner.build(copyBuffer);
                 }
@@ -130,8 +131,9 @@
         private static final long serialVersionUID = 1L;
 
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
-                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
+                final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
+                int nPartitions) {
             IOperatorNodePushable op = new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
                 private InMemoryHashJoin joiner;
 
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/DeserializedMapperOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/DeserializedMapperOperatorDescriptor.java
index 64b45d7..37b5165 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/DeserializedMapperOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/DeserializedMapperOperatorDescriptor.java
@@ -14,7 +14,7 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.map;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -69,7 +69,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new DeserializedOperatorNodePushable(ctx, new MapperOperator(),
                 recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/ConstantTupleSourceOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorDescriptor.java
similarity index 89%
rename from hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/ConstantTupleSourceOperatorDescriptor.java
rename to hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorDescriptor.java
index 10a3252..4ec6393 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/ConstantTupleSourceOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorDescriptor.java
@@ -12,9 +12,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.storage.am.btree.dataflow;
+package edu.uci.ics.hyracks.dataflow.std.misc;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -41,9 +41,8 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
         return new ConstantTupleSourceOperatorNodePushable(ctx, fieldSlots, tupleData, tupleSize);
     }
-
-}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/ConstantTupleSourceOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
similarity index 83%
rename from hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/ConstantTupleSourceOperatorNodePushable.java
rename to hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
index 775c9aa..8784d4a 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/ConstantTupleSourceOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
@@ -13,25 +13,24 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.hyracks.storage.am.btree.dataflow;
+package edu.uci.ics.hyracks.dataflow.std.misc;
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
 
 public class ConstantTupleSourceOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
-
-    private IHyracksContext ctx;
+    private IHyracksStageletContext ctx;
 
     private int[] fieldSlots;
     private byte[] tupleData;
     private int tupleSize;
 
-    public ConstantTupleSourceOperatorNodePushable(IHyracksContext ctx, int[] fieldSlots, byte[] tupleData,
+    public ConstantTupleSourceOperatorNodePushable(IHyracksStageletContext ctx, int[] fieldSlots, byte[] tupleData,
             int tupleSize) {
         super();
         this.fieldSlots = fieldSlots;
@@ -42,12 +41,12 @@
 
     @Override
     public void initialize() throws HyracksDataException {
-        ByteBuffer writeBuffer = ctx.getResourceManager().allocateFrame();
-        FrameTupleAppender appender = new FrameTupleAppender(ctx);
+        ByteBuffer writeBuffer = ctx.allocateFrame();
+        FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
         appender.reset(writeBuffer, true);
         if (fieldSlots != null && tupleData != null && tupleSize > 0)
             appender.append(fieldSlots, tupleData, 0, tupleSize);
         FrameUtils.flushFrame(writeBuffer, writer);
         writer.close();
     }
-}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializingOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializingOperatorDescriptor.java
index 2869f13..23c6385 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializingOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializingOperatorDescriptor.java
@@ -14,21 +14,20 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.misc;
 
-import java.io.File;
-import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileReader;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractActivityNode;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
@@ -37,7 +36,6 @@
 public class MaterializingOperatorDescriptor extends AbstractOperatorDescriptor {
     private static final long serialVersionUID = 1L;
     protected static final String MATERIALIZED_FILE = "materialized-file";
-    protected static final String FRAME_COUNT = "frame-count";
 
     public MaterializingOperatorDescriptor(JobSpecification spec, RecordDescriptor recordDescriptor) {
         super(spec, 1, 1);
@@ -62,49 +60,29 @@
         private static final long serialVersionUID = 1L;
 
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
-                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
+                final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
+                int nPartitions) {
             return new AbstractUnaryInputSinkOperatorNodePushable() {
-                private FileChannel out;
-                private int frameCount;
+                private RunFileWriter out;
 
                 @Override
                 public void open() throws HyracksDataException {
-                    File outFile;
-                    try {
-                        outFile = ctx.getResourceManager().createFile("mat", ".dat");
-                        out = new RandomAccessFile(outFile, "rw").getChannel();
-                    } catch (IOException e) {
-                        throw new HyracksDataException(e);
-                    }
-                    env.set(MATERIALIZED_FILE, outFile.getAbsolutePath());
+                    FileReference file = ctx.getJobletContext().createWorkspaceFile(
+                            MaterializingOperatorDescriptor.class.getSimpleName());
+                    out = new RunFileWriter(file, ctx.getIOManager());
+                    out.open();
                 }
 
                 @Override
                 public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-                    ++frameCount;
-                    buffer.position(0);
-                    buffer.limit(buffer.capacity());
-                    int rem = buffer.capacity();
-                    while (rem > 0) {
-                        int c;
-                        try {
-                            c = out.write(buffer);
-                        } catch (IOException e) {
-                            throw new HyracksDataException(e);
-                        }
-                        rem -= c;
-                    }
+                    out.nextFrame(buffer);
                 }
 
                 @Override
                 public void close() throws HyracksDataException {
-                    try {
-                        env.set(FRAME_COUNT, frameCount);
-                        out.close();
-                    } catch (IOException e) {
-                        throw new HyracksDataException(e);
-                    }
+                    out.close();
+                    env.set(MATERIALIZED_FILE, out);
                 }
 
                 @Override
@@ -123,32 +101,24 @@
         private static final long serialVersionUID = 1L;
 
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
-                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
+                final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
+                int nPartitions) {
             return new AbstractUnaryOutputSourceOperatorNodePushable() {
                 @Override
                 public void initialize() throws HyracksDataException {
-                    try {
-                        File inFile = new File((String) env.get(MATERIALIZED_FILE));
-                        int frameCount = (Integer) env.get(FRAME_COUNT);
-                        FileChannel in = new RandomAccessFile(inFile, "r").getChannel();
-                        ByteBuffer frame = ctx.getResourceManager().allocateFrame();
-                        writer.open();
-                        for (int i = 0; i < frameCount; ++i) {
-                            frame.clear();
-                            int rem = frame.capacity();
-                            while (rem > 0) {
-                                int c = in.read(frame);
-                                rem -= c;
-                            }
-                            frame.flip();
-                            writer.nextFrame(frame);
-                        }
-                        writer.close();
-                        in.close();
-                    } catch (IOException e) {
-                        throw new HyracksDataException(e);
+                    ByteBuffer frame = ctx.allocateFrame();
+                    RunFileWriter out = (RunFileWriter) env.get(MATERIALIZED_FILE);
+                    RunFileReader in = out.createReader();
+                    writer.open();
+                    in.open();
+                    while (in.nextFrame(frame)) {
+                        frame.flip();
+                        writer.nextFrame(frame);
+                        frame.clear();
                     }
+                    in.close();
+                    writer.close();
                 }
 
                 @Override
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/NullSinkOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/NullSinkOperatorDescriptor.java
index c5149a4..4bc4a7f 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/NullSinkOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/NullSinkOperatorDescriptor.java
@@ -16,7 +16,7 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -33,7 +33,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new AbstractUnaryInputSinkOperatorNodePushable() {
             @Override
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/PrinterOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/PrinterOperatorDescriptor.java
index ee21c2a..7f097e0 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/PrinterOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/PrinterOperatorDescriptor.java
@@ -14,7 +14,7 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.misc;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -57,7 +57,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new DeserializedOperatorNodePushable(ctx, new PrinterOperator(),
                 recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitVectorOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitVectorOperatorDescriptor.java
index 8fd10f6..5b1acd5 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitVectorOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitVectorOperatorDescriptor.java
@@ -17,7 +17,7 @@
 import java.util.ArrayList;
 import java.util.List;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
 import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
 import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
@@ -48,7 +48,7 @@
         }
 
         @Override
-        public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, final IOperatorEnvironment env,
+        public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, final IOperatorEnvironment env,
                 IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
             IOpenableDataWriterOperator op = new IOpenableDataWriterOperator() {
                 private ArrayList<Object[]> buffer;
@@ -88,7 +88,7 @@
         }
 
         @Override
-        public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, final IOperatorEnvironment env,
+        public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, final IOperatorEnvironment env,
                 IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
             IOpenableDataWriterOperator op = new IOpenableDataWriterOperator() {
                 private IOpenableDataWriter<Object[]> writer;
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
index 36a49ee..fd1d96e 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
@@ -14,11 +14,10 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.sort;
 
-import java.io.File;
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
@@ -29,6 +28,7 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileReader;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractActivityNode;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
@@ -86,8 +86,9 @@
         }
 
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
-                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
+                final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
+                int nPartitions) {
             final ExternalSortRunGenerator runGen = new ExternalSortRunGenerator(ctx, sortFields,
                     firstKeyNormalizerFactory, comparatorFactories, recordDescriptors[0], framesLimit);
             IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
@@ -126,12 +127,13 @@
         }
 
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
-                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
+                final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
+                int nPartitions) {
             IOperatorNodePushable op = new AbstractUnaryOutputSourceOperatorNodePushable() {
                 @Override
                 public void initialize() throws HyracksDataException {
-                    List<File> runs = (List<File>) env.get(RUNS);
+                    List<RunFileReader> runs = (List<RunFileReader>) env.get(RUNS);
                     FrameSorter frameSorter = (FrameSorter) env.get(FRAMESORTER);
                     ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, frameSorter, runs, sortFields,
                             comparatorFactories, recordDescriptors[0], framesLimit, writer);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
index e2a08c2..50b968a 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
@@ -14,31 +14,32 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.sort;
 
-import java.io.File;
-import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.LinkedList;
 import java.util.List;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileReader;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
 
 public class ExternalSortRunGenerator implements IFrameWriter {
-    private final IHyracksContext ctx;
+    private final IHyracksStageletContext ctx;
     private final FrameSorter frameSorter;
-    private final List<File> runs;
+    private final List<RunFileReader> runs;
     private final int maxSortFrames;
 
-    public ExternalSortRunGenerator(IHyracksContext ctx, int[] sortFields,
+    public ExternalSortRunGenerator(IHyracksStageletContext ctx, int[] sortFields,
             INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
             RecordDescriptor recordDesc, int framesLimit) {
         this.ctx = ctx;
         frameSorter = new FrameSorter(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDesc);
-        runs = new LinkedList<File>();
+        runs = new LinkedList<RunFileReader>();
         maxSortFrames = framesLimit - 1;
     }
 
@@ -69,13 +70,8 @@
 
     private void flushFramesToRun() throws HyracksDataException {
         frameSorter.sortFrames();
-        File runFile;
-        try {
-            runFile = ctx.getResourceManager().createFile(ExternalSortOperatorDescriptor.class.getSimpleName(), ".run");
-        } catch (IOException e) {
-            throw new HyracksDataException(e);
-        }
-        RunFileWriter writer = new RunFileWriter(runFile);
+        FileReference file = ctx.getJobletContext().createWorkspaceFile(ExternalSortRunGenerator.class.getSimpleName());
+        RunFileWriter writer = new RunFileWriter(file, ctx.getIOManager());
         writer.open();
         try {
             frameSorter.flushFrames(writer);
@@ -83,7 +79,7 @@
             writer.close();
         }
         frameSorter.reset();
-        runs.add(runFile);
+        runs.add(writer.createReader());
     }
 
     @Override
@@ -94,7 +90,7 @@
         return frameSorter;
     }
 
-    public List<File> getRuns() {
+    public List<RunFileReader> getRuns() {
         return runs;
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
index 87c021b..19f3b7c 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
@@ -14,8 +14,6 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.sort;
 
-import java.io.File;
-import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Comparator;
@@ -23,21 +21,24 @@
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileReader;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
 import edu.uci.ics.hyracks.dataflow.std.util.ReferenceEntry;
 import edu.uci.ics.hyracks.dataflow.std.util.ReferencedPriorityQueue;
 
 public class ExternalSortRunMerger {
-    private final IHyracksContext ctx;
+    private final IHyracksStageletContext ctx;
     private final FrameSorter frameSorter;
-    private final List<File> runs;
+    private final List<RunFileReader> runs;
     private final int[] sortFields;
     private final IBinaryComparator[] comparators;
     private final RecordDescriptor recordDesc;
@@ -47,9 +48,9 @@
     private ByteBuffer outFrame;
     private FrameTupleAppender outFrameAppender;
 
-    public ExternalSortRunMerger(IHyracksContext ctx, FrameSorter frameSorter, List<File> runs, int[] sortFields,
-            IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDesc, int framesLimit,
-            IFrameWriter writer) {
+    public ExternalSortRunMerger(IHyracksStageletContext ctx, FrameSorter frameSorter, List<RunFileReader> runs,
+            int[] sortFields, IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDesc,
+            int framesLimit, IFrameWriter writer) {
         this.ctx = ctx;
         this.frameSorter = frameSorter;
         this.runs = runs;
@@ -74,11 +75,11 @@
                 }
             } else {
                 inFrames = new ArrayList<ByteBuffer>();
-                outFrame = ctx.getResourceManager().allocateFrame();
-                outFrameAppender = new FrameTupleAppender(ctx);
+                outFrame = ctx.allocateFrame();
+                outFrameAppender = new FrameTupleAppender(ctx.getFrameSize());
                 outFrameAppender.reset(outFrame, true);
                 for (int i = 0; i < framesLimit - 1; ++i) {
-                    inFrames.add(ctx.getResourceManager().allocateFrame());
+                    inFrames.add(ctx.allocateFrame());
                 }
                 int passCount = 0;
                 while (runs.size() > 0) {
@@ -102,8 +103,8 @@
     }
 
     // creates a new run from runs that can fit in memory.
-    private void doPass(List<File> runs, int passCount, boolean doFinalPass) throws HyracksDataException, IOException {
-        File newRun = null;
+    private void doPass(List<RunFileReader> runs, int passCount, boolean doFinalPass) throws HyracksDataException {
+        FileReference newRun = null;
         IFrameWriter writer = this.writer;
         boolean finalPass = false;
         if (runs.size() + 1 <= framesLimit) { // + 1 outFrame
@@ -115,24 +116,24 @@
                 inFrames.remove(i);
             }
         } else {
-            newRun = ctx.getResourceManager().createFile(ExternalSortOperatorDescriptor.class.getSimpleName(), ".run");
-            writer = new RunFileWriter(newRun);
+            newRun = ctx.createWorkspaceFile(ExternalSortRunMerger.class.getSimpleName());
+            writer = new RunFileWriter(newRun, ctx.getIOManager());
             writer.open();
         }
         try {
             RunFileReader[] runCursors = new RunFileReader[inFrames.size()];
             FrameTupleAccessor[] tupleAccessors = new FrameTupleAccessor[inFrames.size()];
             Comparator<ReferenceEntry> comparator = createEntryComparator(comparators);
-            ReferencedPriorityQueue topTuples = new ReferencedPriorityQueue(ctx, recordDesc, inFrames.size(),
-                    comparator);
+            ReferencedPriorityQueue topTuples = new ReferencedPriorityQueue(ctx.getFrameSize(), recordDesc,
+                    inFrames.size(), comparator);
             int[] tupleIndexes = new int[inFrames.size()];
             for (int i = 0; i < inFrames.size(); i++) {
                 tupleIndexes[i] = 0;
                 int runIndex = topTuples.peek().getRunid();
-                runCursors[runIndex] = new RunFileReader(runs.get(runIndex));
+                runCursors[runIndex] = runs.get(runIndex);
                 runCursors[runIndex].open();
                 if (runCursors[runIndex].nextFrame(inFrames.get(runIndex))) {
-                    tupleAccessors[runIndex] = new FrameTupleAccessor(ctx, recordDesc);
+                    tupleAccessors[runIndex] = new FrameTupleAccessor(ctx.getFrameSize(), recordDesc);
                     tupleAccessors[runIndex].reset(inFrames.get(runIndex));
                     setNextTopTuple(runIndex, tupleIndexes, runCursors, tupleAccessors, topTuples);
                 } else {
@@ -163,7 +164,7 @@
             }
             runs.subList(0, inFrames.size()).clear();
             if (!finalPass) {
-                runs.add(0, newRun);
+                runs.add(0, ((RunFileWriter) writer).createReader());
             }
         } finally {
             if (!finalPass) {
@@ -173,7 +174,7 @@
     }
 
     private void setNextTopTuple(int runIndex, int[] tupleIndexes, RunFileReader[] runCursors,
-            FrameTupleAccessor[] tupleAccessors, ReferencedPriorityQueue topTuples) throws IOException {
+            FrameTupleAccessor[] tupleAccessors, ReferencedPriorityQueue topTuples) throws HyracksDataException {
         boolean exists = hasNextTuple(runIndex, tupleIndexes, runCursors, tupleAccessors);
         if (exists) {
             topTuples.popAndReplace(tupleAccessors[runIndex], tupleIndexes[runIndex]);
@@ -184,7 +185,7 @@
     }
 
     private boolean hasNextTuple(int runIndex, int[] tupleIndexes, RunFileReader[] runCursors,
-            FrameTupleAccessor[] tupleAccessors) throws IOException {
+            FrameTupleAccessor[] tupleAccessors) throws HyracksDataException {
         if (tupleAccessors[runIndex] == null || runCursors[runIndex] == null) {
             return false;
         } else if (tupleIndexes[runIndex] >= tupleAccessors[runIndex].getTupleCount()) {
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
index 0556516..603b194 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
@@ -19,7 +19,7 @@
 import java.util.List;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
@@ -31,7 +31,7 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 
 public class FrameSorter {
-    private final IHyracksContext ctx;
+    private final IHyracksCommonContext ctx;
     private final int[] sortFields;
     private final INormalizedKeyComputer nkc;
     private final IBinaryComparator[] comparators;
@@ -44,8 +44,9 @@
     private int dataFrameCount;
     private int[] tPointers;
 
-    public FrameSorter(IHyracksContext ctx, int[] sortFields, INormalizedKeyComputerFactory firstKeyNormalizerFactory,
-            IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor) {
+    public FrameSorter(IHyracksCommonContext ctx, int[] sortFields,
+            INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDescriptor) {
         this.ctx = ctx;
         this.sortFields = sortFields;
         nkc = firstKeyNormalizerFactory == null ? null : firstKeyNormalizerFactory.createNormalizedKeyComputer();
@@ -55,8 +56,8 @@
         }
         this.recordDescriptor = recordDescriptor;
         buffers = new ArrayList<ByteBuffer>();
-        fta1 = new FrameTupleAccessor(ctx, recordDescriptor);
-        fta2 = new FrameTupleAccessor(ctx, recordDescriptor);
+        fta1 = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
+        fta2 = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
 
         dataFrameCount = 0;
     }
@@ -89,7 +90,7 @@
     public void insertFrame(ByteBuffer buffer) {
         ByteBuffer copyFrame;
         if (dataFrameCount == buffers.size()) {
-            copyFrame = ctx.getResourceManager().allocateFrame();
+            copyFrame = ctx.allocateFrame();
             buffers.add(copyFrame);
         } else {
             copyFrame = buffers.get(dataFrameCount);
@@ -99,7 +100,7 @@
     }
 
     public void sortFrames() {
-        FrameTupleAccessor accessor = new FrameTupleAccessor(ctx, recordDescriptor);
+        FrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
         int nBuffers = dataFrameCount;
         int totalTCount = 0;
         for (int i = 0; i < nBuffers; ++i) {
@@ -132,9 +133,9 @@
     }
 
     public void flushFrames(IFrameWriter writer) throws HyracksDataException {
-        FrameTupleAccessor accessor = new FrameTupleAccessor(ctx, recordDescriptor);
-        FrameTupleAppender appender = new FrameTupleAppender(ctx);
-        ByteBuffer outFrame = ctx.getResourceManager().allocateFrame();
+        FrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
+        FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+        ByteBuffer outFrame = ctx.allocateFrame();
         writer.open();
         appender.reset(outFrame, true);
         int n = tPointers.length / 4;
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
index 7f6baa7..3d41e18 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
@@ -16,7 +16,7 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
@@ -78,8 +78,9 @@
         }
 
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
-                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
+                final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
+                int nPartitions) {
             final FrameSorter frameSorter = new FrameSorter(ctx, sortFields, firstKeyNormalizerFactory,
                     comparatorFactories, recordDescriptors[0]);
             IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
@@ -116,8 +117,9 @@
         }
 
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
-                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
+                final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
+                int nPartitions) {
             IOperatorNodePushable op = new AbstractUnaryOutputSourceOperatorNodePushable() {
                 @Override
                 public void initialize() throws HyracksDataException {
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunFileReader.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunFileReader.java
deleted file mode 100644
index f31100e..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunFileReader.java
+++ /dev/null
@@ -1,59 +0,0 @@
-package edu.uci.ics.hyracks.dataflow.std.sort;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-
-public class RunFileReader implements IFrameReader {
-    private final File file;
-    private FileChannel channel;
-
-    public RunFileReader(File file) throws FileNotFoundException {
-        this.file = file;
-    }
-
-    @Override
-    public void open() throws HyracksDataException {
-        RandomAccessFile raf;
-        try {
-            raf = new RandomAccessFile(file, "r");
-        } catch (FileNotFoundException e) {
-            throw new HyracksDataException(e);
-        }
-        channel = raf.getChannel();
-    }
-
-    @Override
-    public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        buffer.clear();
-        int remain = buffer.capacity();
-        while (remain > 0) {
-            int len;
-            try {
-                len = channel.read(buffer);
-            } catch (IOException e) {
-                throw new HyracksDataException(e);
-            }
-            if (len < 0) {
-                return false;
-            }
-            remain -= len;
-        }
-        return true;
-    }
-
-    @Override
-    public void close() throws HyracksDataException {
-        try {
-            channel.close();
-        } catch (IOException e) {
-            throw new HyracksDataException(e);
-        }
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunFileWriter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunFileWriter.java
deleted file mode 100644
index fd0a03e..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunFileWriter.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed 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 from
- * 
- *     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 edu.uci.ics.hyracks.dataflow.std.sort;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-
-public class RunFileWriter implements IFrameWriter {
-    private final File file;
-    private FileChannel channel;
-
-    public RunFileWriter(File file) {
-        this.file = file;
-    }
-
-    @Override
-    public void open() throws HyracksDataException {
-        RandomAccessFile raf;
-        try {
-            raf = new RandomAccessFile(file, "rw");
-        } catch (FileNotFoundException e) {
-            throw new HyracksDataException(e);
-        }
-        channel = raf.getChannel();
-    }
-
-    @Override
-    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        int remain = buffer.capacity();
-        while (remain > 0) {
-            int len;
-            try {
-                len = channel.write(buffer);
-            } catch (IOException e) {
-                throw new HyracksDataException(e);
-            }
-            if (len < 0) {
-                throw new HyracksDataException("Error writing data");
-            }
-            remain -= len;
-        }
-    }
-
-    @Override
-    public void close() throws HyracksDataException {
-        if (channel != null) {
-            try {
-                channel.close();
-            } catch (IOException e) {
-                throw new HyracksDataException(e);
-            }
-        }
-    }
-
-    @Override
-    public void flush() throws HyracksDataException {
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java
index 863d531..6660a60 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java
@@ -17,7 +17,7 @@
 import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameDeserializer;
@@ -26,17 +26,17 @@
 import edu.uci.ics.hyracks.dataflow.std.base.IOpenableDataWriterOperator;
 
 public final class DeserializedOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable {
-    private final IHyracksContext ctx;
+    private final IHyracksStageletContext ctx;
 
     private final IOpenableDataWriterOperator delegate;
 
     private final FrameDeserializer deserializer;
 
-    public DeserializedOperatorNodePushable(IHyracksContext ctx, IOpenableDataWriterOperator delegate,
+    public DeserializedOperatorNodePushable(IHyracksStageletContext ctx, IOpenableDataWriterOperator delegate,
             RecordDescriptor inRecordDesc) {
         this.ctx = ctx;
         this.delegate = delegate;
-        deserializer = inRecordDesc == null ? null : new FrameDeserializer(ctx, inRecordDesc);
+        deserializer = inRecordDesc == null ? null : new FrameDeserializer(ctx.getFrameSize(), inRecordDesc);
     }
 
     @Override
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java
index e37c51c..b209a4a 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java
@@ -18,12 +18,11 @@
 import java.util.BitSet;
 import java.util.Comparator;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 
 public class ReferencedPriorityQueue {
-    private final IHyracksContext ctx;
+    private final int frameSize;
     private final RecordDescriptor recordDescriptor;
     private final ReferenceEntry entries[];
     private final int size;
@@ -32,9 +31,9 @@
 
     private final Comparator<ReferenceEntry> comparator;
 
-    public ReferencedPriorityQueue(IHyracksContext ctx, RecordDescriptor recordDescriptor, int initSize,
-            Comparator<ReferenceEntry> comparator) throws IOException {
-        this.ctx = ctx;
+    public ReferencedPriorityQueue(int frameSize, RecordDescriptor recordDescriptor, int initSize,
+            Comparator<ReferenceEntry> comparator) {
+        this.frameSize = frameSize;
         this.recordDescriptor = recordDescriptor;
         if (initSize < 1)
             throw new IllegalArgumentException();
@@ -66,10 +65,10 @@
      * @return runid of this entry
      * @throws IOException
      */
-    public int popAndReplace(FrameTupleAccessor fta, int tIndex) throws IOException {
+    public int popAndReplace(FrameTupleAccessor fta, int tIndex) {
         ReferenceEntry entry = entries[0];
         if (entry.getAccessor() == null) {
-            entry.setAccessor(new FrameTupleAccessor(ctx, recordDescriptor));
+            entry.setAccessor(new FrameTupleAccessor(frameSize, recordDescriptor));
         }
         entry.getAccessor().reset(fta.getBuffer());
         entry.setTupleIndex(tIndex);
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
index f293a1f..07fd5ba 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
@@ -46,7 +46,7 @@
 import edu.uci.ics.hyracks.dataflow.std.misc.NullSinkOperatorDescriptor;
 import edu.uci.ics.hyracks.examples.btree.helper.BTreeRegistryProvider;
 import edu.uci.ics.hyracks.examples.btree.helper.DataGenOperatorDescriptor;
-import edu.uci.ics.hyracks.examples.btree.helper.SimpleStorageManager;
+import edu.uci.ics.hyracks.examples.btree.helper.StorageManagerInterface;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeInsertUpdateDeleteOperatorDescriptor;
@@ -133,7 +133,7 @@
         dataGen.setPartitionConstraint(dataGenConstraint);
 
         IBTreeRegistryProvider btreeRegistryProvider = BTreeRegistryProvider.INSTANCE;
-        IStorageManagerInterface storageManager = SimpleStorageManager.INSTANCE;
+        IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
 
         // prepare insertion into primary index
         // tuples to be put into B-Tree shall have 4 fields
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/JobHelper.java b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/JobHelper.java
index 7702aa5..af79d32 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/JobHelper.java
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/JobHelper.java
@@ -21,6 +21,7 @@
 import edu.uci.ics.hyracks.api.constraints.ExplicitPartitionConstraint;
 import edu.uci.ics.hyracks.api.constraints.LocationConstraint;
 import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
+import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
@@ -30,7 +31,7 @@
         FileSplit[] fileSplits = new FileSplit[splitNCs.length];
         for (int i = 0; i < splitNCs.length; ++i) {
             String fileName = btreeFileName + "." + splitNCs[i];
-            fileSplits[i] = new FileSplit(splitNCs[i], new File(fileName));
+            fileSplits[i] = new FileSplit(splitNCs[i], new FileReference(new File(fileName)));
         }
         IFileSplitProvider splitProvider = new ConstantFileSplitProvider(fileSplits);
         return splitProvider;
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
index 1fc1604..336efb3 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
@@ -44,7 +44,7 @@
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.examples.btree.helper.BTreeRegistryProvider;
 import edu.uci.ics.hyracks.examples.btree.helper.DataGenOperatorDescriptor;
-import edu.uci.ics.hyracks.examples.btree.helper.SimpleStorageManager;
+import edu.uci.ics.hyracks.examples.btree.helper.StorageManagerInterface;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeBulkLoadOperatorDescriptor;
@@ -153,7 +153,7 @@
         IBTreeInteriorFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(tupleWriterFactory);
         IBTreeLeafFrameFactory leafFrameFactory = new NSMLeafFrameFactory(tupleWriterFactory);
         IBTreeRegistryProvider btreeRegistryProvider = BTreeRegistryProvider.INSTANCE;
-        IStorageManagerInterface storageManager = SimpleStorageManager.INSTANCE;
+        IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
 
         // the B-Tree expects its keyfields to be at the front of its input
         // tuple
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexEnlistFilesExample.java b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexEnlistFilesExample.java
index 9478c78..fedf0bf 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexEnlistFilesExample.java
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexEnlistFilesExample.java
@@ -33,7 +33,7 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.examples.btree.helper.BTreeRegistryProvider;
-import edu.uci.ics.hyracks.examples.btree.helper.SimpleStorageManager;
+import edu.uci.ics.hyracks.examples.btree.helper.StorageManagerInterface;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeFileEnlistmentOperatorDescriptor;
@@ -103,7 +103,7 @@
         IBTreeInteriorFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(tupleWriterFactory);
         IBTreeLeafFrameFactory leafFrameFactory = new NSMLeafFrameFactory(tupleWriterFactory);
         IBTreeRegistryProvider btreeRegistryProvider = BTreeRegistryProvider.INSTANCE;
-        IStorageManagerInterface storageManager = SimpleStorageManager.INSTANCE;
+        IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
 
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
         comparatorFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
index 06b5bc9..fe9bee2 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
@@ -36,13 +36,13 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.PrinterOperatorDescriptor;
 import edu.uci.ics.hyracks.examples.btree.helper.BTreeRegistryProvider;
-import edu.uci.ics.hyracks.examples.btree.helper.SimpleStorageManager;
+import edu.uci.ics.hyracks.examples.btree.helper.StorageManagerInterface;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.ConstantTupleSourceOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.IBTreeRegistryProvider;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMLeafFrameFactory;
@@ -105,7 +105,7 @@
         IBTreeInteriorFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(tupleWriterFactory);
         IBTreeLeafFrameFactory leafFrameFactory = new NSMLeafFrameFactory(tupleWriterFactory);
         IBTreeRegistryProvider btreeRegistryProvider = BTreeRegistryProvider.INSTANCE;
-        IStorageManagerInterface storageManager = SimpleStorageManager.INSTANCE;
+        IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
 
         // schema of tuples coming out of primary index
         RecordDescriptor recDesc = new RecordDescriptor(new ISerializerDeserializer[] {
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
index 2c4d8fb..403e0a9 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
@@ -36,7 +36,7 @@
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.examples.btree.helper.BTreeRegistryProvider;
-import edu.uci.ics.hyracks.examples.btree.helper.SimpleStorageManager;
+import edu.uci.ics.hyracks.examples.btree.helper.StorageManagerInterface;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeBulkLoadOperatorDescriptor;
@@ -98,7 +98,7 @@
         String[] splitNCs = options.ncs.split(",");
 
         IBTreeRegistryProvider btreeRegistryProvider = BTreeRegistryProvider.INSTANCE;
-        IStorageManagerInterface storageManager = SimpleStorageManager.INSTANCE;
+        IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
 
         // schema of tuples that we are retrieving from the primary index
         RecordDescriptor recDesc = new RecordDescriptor(new ISerializerDeserializer[] {
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
index 6fd39f0..ba2c72f 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
@@ -36,13 +36,13 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.PrinterOperatorDescriptor;
 import edu.uci.ics.hyracks.examples.btree.helper.BTreeRegistryProvider;
-import edu.uci.ics.hyracks.examples.btree.helper.SimpleStorageManager;
+import edu.uci.ics.hyracks.examples.btree.helper.StorageManagerInterface;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.ConstantTupleSourceOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.IBTreeRegistryProvider;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMLeafFrameFactory;
@@ -97,7 +97,7 @@
         String[] splitNCs = options.ncs.split(",");
 
         IBTreeRegistryProvider btreeRegistryProvider = BTreeRegistryProvider.INSTANCE;
-        IStorageManagerInterface storageManager = SimpleStorageManager.INSTANCE;
+        IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
 
         // schema of tuples coming out of secondary index
         RecordDescriptor secondaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
diff --git a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/BTreeRegistryProvider.java b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/BTreeRegistryProvider.java
index 46ef79f..57b10a3 100644
--- a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/BTreeRegistryProvider.java
+++ b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/BTreeRegistryProvider.java
@@ -15,11 +15,11 @@
 
 package edu.uci.ics.hyracks.examples.btree.helper;
 
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeRegistry;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.IBTreeRegistryProvider;
 
 public class BTreeRegistryProvider implements IBTreeRegistryProvider {
-
     private static final long serialVersionUID = 1L;
 
     public static final BTreeRegistryProvider INSTANCE = new BTreeRegistryProvider();
@@ -28,7 +28,7 @@
     }
 
     @Override
-    public BTreeRegistry getBTreeRegistry() {
-        return RuntimeContext.getInstance().getBTreeRegistry();
+    public BTreeRegistry getBTreeRegistry(IHyracksStageletContext ctx) {
+        return RuntimeContext.get(ctx).getBTreeRegistry();
     }
-}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
index a325739..0c72503 100644
--- a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
+++ b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
@@ -20,7 +20,7 @@
 import java.util.HashSet;
 import java.util.Random;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -59,11 +59,11 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
 
-        final ByteBuffer outputFrame = ctx.getResourceManager().allocateFrame();
-        final FrameTupleAppender appender = new FrameTupleAppender(ctx);
+        final ByteBuffer outputFrame = ctx.allocateFrame();
+        final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
         final RecordDescriptor recDesc = recordDescriptors[0];
         final ArrayTupleBuilder tb = new ArrayTupleBuilder(recDesc.getFields().length);
         final Random rnd = new Random(randomSeed);
diff --git a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/NCBootstrap.java b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/NCBootstrap.java
index d56aef7..ea55e7a 100644
--- a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/NCBootstrap.java
+++ b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/NCBootstrap.java
@@ -28,14 +28,16 @@
     @Override
     public void start() throws Exception {
         LOGGER.info("Starting NC Bootstrap");
-        RuntimeContext.initialize();
-        LOGGER.info("Initialized RuntimeContext: " + RuntimeContext.getInstance());
+        RuntimeContext rCtx = new RuntimeContext(appCtx);
+        appCtx.setApplicationObject(rCtx);
+        LOGGER.info("Initialized RuntimeContext: " + rCtx);
     }
 
     @Override
     public void stop() throws Exception {
         LOGGER.info("Stopping Asterix NC Bootstrap");
-        RuntimeContext.deinitialize();
+        RuntimeContext rCtx = (RuntimeContext) appCtx.getApplicationObject();
+        rCtx.close();
     }
 
     @Override
diff --git a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java
index 9cd55ed..9f99468 100644
--- a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java
+++ b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java
@@ -15,6 +15,8 @@
 
 package edu.uci.ics.hyracks.examples.btree.helper;
 
+import edu.uci.ics.hyracks.api.application.INCApplicationContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeRegistry;
 import edu.uci.ics.hyracks.storage.common.buffercache.BufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ClockPageReplacementStrategy;
@@ -24,46 +26,23 @@
 import edu.uci.ics.hyracks.storage.common.buffercache.IPageReplacementStrategy;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapManager;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.storage.common.smi.TransientFileMapManager;
 
 public class RuntimeContext {
-    private static RuntimeContext INSTANCE;
-
     private BTreeRegistry btreeRegistry;
     private IBufferCache bufferCache;
     private IFileMapManager fileMapManager;
 
-    private RuntimeContext() {
-    }
-
-    public static void initialize() {
-        if (INSTANCE != null) {
-            throw new IllegalStateException("Instance already initialized");
-        }
-        INSTANCE = new RuntimeContext();
-        INSTANCE.start();
-    }
-
-    public static void deinitialize() {
-        if (INSTANCE != null) {
-            INSTANCE.stop();
-            INSTANCE = null;
-        }
-    }
-
-    private void stop() {
-        bufferCache.close();
-    }
-
-    private void start() {
-        fileMapManager = new SimpleFileMapManager();
+    public RuntimeContext(INCApplicationContext appCtx) {
+        fileMapManager = new TransientFileMapManager();
         ICacheMemoryAllocator allocator = new HeapBufferAllocator();
         IPageReplacementStrategy prs = new ClockPageReplacementStrategy();
-        bufferCache = new BufferCache(allocator, prs, fileMapManager, 32768, 50);
+        bufferCache = new BufferCache(appCtx.getRootContext().getIOManager(), allocator, prs, fileMapManager, 32768, 50);
         btreeRegistry = new BTreeRegistry();
     }
 
-    public static RuntimeContext getInstance() {
-        return INSTANCE;
+    public void close() {
+        bufferCache.close();
     }
 
     public IBufferCache getBufferCache() {
@@ -77,4 +56,8 @@
     public BTreeRegistry getBTreeRegistry() {
         return btreeRegistry;
     }
+    
+    public static RuntimeContext get(IHyracksStageletContext ctx) {
+        return (RuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject();
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/SimpleFileMapManager.java b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/SimpleFileMapManager.java
deleted file mode 100644
index 21f864e..0000000
--- a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/SimpleFileMapManager.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed 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 from
- * 
- *     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 edu.uci.ics.hyracks.examples.btree.helper;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.common.file.IFileMapManager;
-
-public class SimpleFileMapManager implements IFileMapManager {
-
-    private static final long serialVersionUID = 1L;
-    private Map<Integer, String> id2nameMap = new HashMap<Integer, String>();
-    private Map<String, Integer> name2IdMap = new HashMap<String, Integer>();
-    private int idCounter = 0;
-
-    @Override
-    public String lookupFileName(int fileId) throws HyracksDataException {
-        String fName = id2nameMap.get(fileId);
-        if (fName == null) {
-            throw new HyracksDataException("No mapping found for id: " + fileId);
-        }
-        return fName;
-    }
-
-    @Override
-    public int lookupFileId(String fileName) throws HyracksDataException {
-        Integer fileId = name2IdMap.get(fileName);
-        if (fileId == null) {
-            throw new HyracksDataException("No mapping found for name: " + fileName);
-        }
-        return fileId;
-    }
-
-    @Override
-    public boolean isMapped(String fileName) {
-        return name2IdMap.containsKey(fileName);
-    }
-
-    @Override
-    public boolean isMapped(int fileId) {
-        return id2nameMap.containsKey(fileId);
-    }
-
-    @Override
-    public void unregisterFile(int fileId) throws HyracksDataException {
-        String fileName = id2nameMap.remove(fileId);
-        name2IdMap.remove(fileName);
-    }
-
-    @Override
-    public void registerFile(String fileName) throws HyracksDataException {
-        Integer fileId = idCounter++;
-        id2nameMap.put(fileId, fileName);
-        name2IdMap.put(fileName, fileId);
-    }
-}
diff --git a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/SimpleStorageManager.java b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/StorageManagerInterface.java
similarity index 64%
copy from hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/SimpleStorageManager.java
copy to hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/StorageManagerInterface.java
index 8348bb9..e1e47af 100644
--- a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/SimpleStorageManager.java
+++ b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/StorageManagerInterface.java
@@ -15,24 +15,26 @@
 
 package edu.uci.ics.hyracks.examples.btree.helper;
 
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 
-public class SimpleStorageManager implements IStorageManagerInterface {
-
+public class StorageManagerInterface implements IStorageManagerInterface {
     private static final long serialVersionUID = 1L;
 
-    public static SimpleStorageManager INSTANCE = new SimpleStorageManager();
+    public static final StorageManagerInterface INSTANCE = new StorageManagerInterface();
 
-    @Override
-    public IBufferCache getBufferCache() {
-        return RuntimeContext.getInstance().getBufferCache();
+    private StorageManagerInterface() {
     }
 
     @Override
-    public IFileMapProvider getFileMapProvider() {
-        return RuntimeContext.getInstance().getFileMapManager();
+    public IBufferCache getBufferCache(IHyracksStageletContext ctx) {
+        return RuntimeContext.get(ctx).getBufferCache();
     }
 
-}
+    @Override
+    public IFileMapProvider getFileMapProvider(IHyracksStageletContext ctx) {
+        return RuntimeContext.get(ctx).getFileMapManager();
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml b/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
index d13001e..8cf1db7 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
@@ -74,5 +74,12 @@
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-test-support</artifactId>
+  		<version>0.1.4-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>test</scope>
+  	</dependency>
   </dependencies>
 </project>
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeOperatorsTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeOperatorsTest.java
index 54b97c1..7f2152e 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeOperatorsTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeOperatorsTest.java
@@ -30,6 +30,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
+import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -43,6 +44,7 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.NullSinkOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.PrinterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.InMemorySortOperatorDescriptor;
@@ -52,10 +54,7 @@
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeBulkLoadOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeInsertUpdateDeleteOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeRegistry;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeRegistryProvider;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.ConstantTupleSourceOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.HyracksSimpleStorageManagerInterface;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.IBTreeRegistryProvider;
 import edu.uci.ics.hyracks.storage.am.btree.frames.MetaDataFrame;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMInteriorFrameFactory;
@@ -67,21 +66,25 @@
 import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
 import edu.uci.ics.hyracks.storage.am.btree.impls.RangeSearchCursor;
 import edu.uci.ics.hyracks.storage.am.btree.tuples.TypeAwareTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.test.support.TestBTreeRegistryProvider;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
 import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
 
 public class BTreeOperatorsTest extends AbstractIntegrationTest {
-
-    private static HyracksSimpleStorageManagerInterface storageManager = new HyracksSimpleStorageManagerInterface(8192,
-            20);
+    static {
+        TestStorageManagerComponentHolder.init(8192, 20);
+    }
 
     @Test
     public void bulkLoadTest() throws Exception {
-
         JobSpecification spec = new JobSpecification();
 
-        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID, new File("data/tpch0.001/orders-part1.tbl")) };
+        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
+                "data/tpch0.001/orders-part1.tbl"))) };
         IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
         RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
@@ -123,14 +126,15 @@
         // SimpleTupleWriterFactory();
         IBTreeInteriorFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(tupleWriterFactory);
         IBTreeLeafFrameFactory leafFrameFactory = new NSMLeafFrameFactory(tupleWriterFactory);
-        IBTreeRegistryProvider btreeRegistryProvider = new BTreeRegistryProvider();
+        IBTreeRegistryProvider btreeRegistryProvider = new TestBTreeRegistryProvider();
 
         int[] fieldPermutation = { 0, 4, 5 };
         String btreeName = "btree.bin";
-        String nc1FileName = System.getProperty("java.io.tmpdir") + "/nc1/" + btreeName;
+        FileReference nc1File = new FileReference(new File(System.getProperty("java.io.tmpdir") + "/nc1/" + btreeName));
         IFileSplitProvider btreeSplitProvider = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                new File(nc1FileName)) });
+                nc1File) });
 
+        IStorageManagerInterface storageManager = new TestStorageManagerInterface();
         BTreeBulkLoadOperatorDescriptor btreeBulkLoad = new BTreeBulkLoadOperatorDescriptor(spec, storageManager,
                 btreeRegistryProvider, btreeSplitProvider, interiorFrameFactory, leafFrameFactory, typeTraits,
                 comparatorFactories, fieldPermutation, 0.7f);
@@ -155,9 +159,9 @@
         MultiComparator cmp = new MultiComparator(typeTraits, comparators);
 
         // try an ordered scan on the bulk-loaded btree
-        int btreeFileId = storageManager.getFileMapProvider().lookupFileId(nc1FileName);
-        storageManager.getBufferCache().openFile(btreeFileId);
-        BTree btree = btreeRegistryProvider.getBTreeRegistry().get(btreeFileId);
+        int btreeFileId = storageManager.getFileMapProvider(null).lookupFileId(nc1File);
+        storageManager.getBufferCache(null).openFile(btreeFileId);
+        BTree btree = btreeRegistryProvider.getBTreeRegistry(null).get(btreeFileId);
         IBTreeCursor scanCursor = new RangeSearchCursor(leafFrameFactory.getFrame());
         RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
         BTreeOpContext opCtx = btree.createOpContext(BTreeOp.BTO_SEARCH, leafFrameFactory.getFrame(),
@@ -175,7 +179,7 @@
         } finally {
             scanCursor.close();
         }
-        storageManager.getBufferCache().closeFile(btreeFileId);
+        storageManager.getBufferCache(null).closeFile(btreeFileId);
     }
 
     @Test
@@ -233,17 +237,18 @@
         PartitionConstraint keyProviderPartitionConstraint = new ExplicitPartitionConstraint(
                 new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
         keyProviderOp.setPartitionConstraint(keyProviderPartitionConstraint);
-        IBTreeRegistryProvider btreeRegistryProvider = new BTreeRegistryProvider();
+        IBTreeRegistryProvider btreeRegistryProvider = new TestBTreeRegistryProvider();
 
         RecordDescriptor recDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
                 UTF8StringSerializerDeserializer.INSTANCE });
 
         String btreeName = "btree.bin";
-        String nc1FileName = System.getProperty("java.io.tmpdir") + "/nc1/" + btreeName;
+        FileReference nc1File = new FileReference(new File(System.getProperty("java.io.tmpdir") + "/nc1/" + btreeName));
         IFileSplitProvider btreeSplitProvider = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                new File(nc1FileName)) });
+                nc1File) });
 
+        IStorageManagerInterface storageManager = new TestStorageManagerInterface();
         BTreeSearchOperatorDescriptor btreeSearchOp = new BTreeSearchOperatorDescriptor(spec, recDesc, storageManager,
                 btreeRegistryProvider, btreeSplitProvider, interiorFrameFactory, leafFrameFactory, typeTraits,
                 comparatorFactories, true, new int[] { 0 }, new int[] { 1 }, true, true);
@@ -275,7 +280,8 @@
 
         JobSpecification spec = new JobSpecification();
 
-        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID, new File("data/tpch0.001/orders-part1.tbl")) };
+        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
+                "data/tpch0.001/orders-part1.tbl"))) };
         IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
         RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
@@ -314,7 +320,7 @@
         // SimpleTupleWriterFactory();
         IBTreeInteriorFrameFactory primaryInteriorFrameFactory = new NSMInteriorFrameFactory(primaryTupleWriterFactory);
         IBTreeLeafFrameFactory primaryLeafFrameFactory = new NSMLeafFrameFactory(primaryTupleWriterFactory);
-        IBTreeRegistryProvider btreeRegistryProvider = new BTreeRegistryProvider();
+        IBTreeRegistryProvider btreeRegistryProvider = new TestBTreeRegistryProvider();
 
         // construct a multicomparator for the primary index
         IBinaryComparator[] primaryComparators = new IBinaryComparator[primaryComparatorFactories.length];
@@ -352,14 +358,15 @@
 
         // we create and register 3 btrees for in an insert pipeline being fed
         // from a filescan op
-        BTreeRegistry btreeRegistry = btreeRegistryProvider.getBTreeRegistry();
-        IBufferCache bufferCache = storageManager.getBufferCache();
-        IFileMapProvider fileMapProvider = storageManager.getFileMapProvider();
+        BTreeRegistry btreeRegistry = btreeRegistryProvider.getBTreeRegistry(null);
+        IStorageManagerInterface storageManager = new TestStorageManagerInterface();
+        IBufferCache bufferCache = storageManager.getBufferCache(null);
+        IFileMapProvider fileMapProvider = storageManager.getFileMapProvider(null);
 
         // primary index
-        String fileNameA = "/tmp/btreetestA.ix";
-        bufferCache.createFile(fileNameA);
-        int fileIdA = fileMapProvider.lookupFileId(fileNameA);
+        FileReference fileA = new FileReference(new File("/tmp/btreetestA.ix"));
+        bufferCache.createFile(fileA);
+        int fileIdA = fileMapProvider.lookupFileId(fileA);
         bufferCache.openFile(fileIdA);
         BTree btreeA = new BTree(bufferCache, primaryInteriorFrameFactory, primaryLeafFrameFactory, primaryCmp);
         btreeA.create(fileIdA, primaryLeafFrameFactory.getFrame(), new MetaDataFrame());
@@ -368,9 +375,9 @@
         bufferCache.closeFile(fileIdA);
 
         // first secondary index
-        String fileNameB = "/tmp/btreetestB.ix";
-        bufferCache.createFile(fileNameB);
-        int fileIdB = fileMapProvider.lookupFileId(fileNameB);
+        FileReference fileB = new FileReference(new File("/tmp/btreetestB.ix"));
+        bufferCache.createFile(fileB);
+        int fileIdB = fileMapProvider.lookupFileId(fileB);
         bufferCache.openFile(fileIdB);
         BTree btreeB = new BTree(bufferCache, secondaryInteriorFrameFactory, secondaryLeafFrameFactory, secondaryCmp);
         btreeB.create(fileIdB, secondaryLeafFrameFactory.getFrame(), new MetaDataFrame());
@@ -379,9 +386,9 @@
         bufferCache.closeFile(fileIdB);
 
         // second secondary index
-        String fileNameC = "/tmp/btreetestC.ix";
-        bufferCache.createFile(fileNameC);
-        int fileIdC = fileMapProvider.lookupFileId(fileNameC);
+        FileReference fileC = new FileReference(new File("/tmp/btreetestC.ix"));
+        bufferCache.createFile(fileC);
+        int fileIdC = fileMapProvider.lookupFileId(fileC);
         bufferCache.openFile(fileIdC);
         BTree btreeC = new BTree(bufferCache, secondaryInteriorFrameFactory, secondaryLeafFrameFactory, secondaryCmp);
         btreeC.create(fileIdC, secondaryLeafFrameFactory.getFrame(), new MetaDataFrame());
@@ -393,7 +400,7 @@
 
         // primary index
         IFileSplitProvider btreeSplitProviderA = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                new File("/tmp/btreetestA.ix")) });
+                new FileReference(new File("/tmp/btreetestA.ix"))) });
         int[] fieldPermutationA = { 0, 1, 2, 3, 4, 5 };
         BTreeInsertUpdateDeleteOperatorDescriptor insertOpA = new BTreeInsertUpdateDeleteOperatorDescriptor(spec,
                 ordersDesc, storageManager, btreeRegistryProvider, btreeSplitProviderA, primaryInteriorFrameFactory,
@@ -405,7 +412,7 @@
 
         // first secondary index
         IFileSplitProvider btreeSplitProviderB = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                new File("/tmp/btreetestB.ix")) });
+                new FileReference(new File("/tmp/btreetestB.ix"))) });
         int[] fieldPermutationB = { 3, 0 };
         BTreeInsertUpdateDeleteOperatorDescriptor insertOpB = new BTreeInsertUpdateDeleteOperatorDescriptor(spec,
                 ordersDesc, storageManager, btreeRegistryProvider, btreeSplitProviderB, secondaryInteriorFrameFactory,
@@ -417,7 +424,7 @@
 
         // second secondary index
         IFileSplitProvider btreeSplitProviderC = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                new File("/tmp/btreetestC.ix")) });
+                new FileReference(new File("/tmp/btreetestC.ix"))) });
         int[] fieldPermutationC = { 4, 0 };
         BTreeInsertUpdateDeleteOperatorDescriptor insertOpC = new BTreeInsertUpdateDeleteOperatorDescriptor(spec,
                 ordersDesc, storageManager, btreeRegistryProvider, btreeSplitProviderC, secondaryInteriorFrameFactory,
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java
index b75fae3..9aac876 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java
@@ -19,35 +19,46 @@
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.UUID;
 
 import org.junit.Test;
 
+import edu.uci.ics.hyracks.api.application.INCApplicationContext;
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
 import edu.uci.ics.hyracks.api.dataflow.IOpenableDataReader;
 import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.control.nc.runtime.RootHyracksContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameDeserializingDataReader;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.SerializingDataWriter;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.test.support.TestJobletContext;
+import edu.uci.ics.hyracks.test.support.TestNCApplicationContext;
+import edu.uci.ics.hyracks.test.support.TestRootContext;
+import edu.uci.ics.hyracks.test.support.TestStageletContext;
 
 public class SerializationDeserializationTest {
     private static final String DBLP_FILE = "data/dblp.txt";
 
     private static class SerDeserRunner {
-        private final IHyracksContext ctx;
+        private final IHyracksStageletContext ctx;
         private static final int FRAME_SIZE = 32768;
         private RecordDescriptor rDes;
         private List<ByteBuffer> buffers;
 
-        public SerDeserRunner(RecordDescriptor rDes) {
-            ctx = new RootHyracksContext(FRAME_SIZE);
+        public SerDeserRunner(RecordDescriptor rDes) throws HyracksException {
+            IHyracksRootContext rootCtx = new TestRootContext(FRAME_SIZE);
+            INCApplicationContext appCtx = new TestNCApplicationContext(rootCtx);
+            IHyracksJobletContext jobletCtx = new TestJobletContext(appCtx, UUID.randomUUID(), 0);
+            ctx = new TestStageletContext(jobletCtx, UUID.randomUUID());
             this.rDes = rDes;
             buffers = new ArrayList<ByteBuffer>();
         }
@@ -60,7 +71,7 @@
 
                 @Override
                 public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-                    ByteBuffer toBuf = ctx.getResourceManager().allocateFrame();
+                    ByteBuffer toBuf = ctx.allocateFrame();
                     toBuf.put(buffer);
                     buffers.add(toBuf);
                 }
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
index 67bcd63..e5d286e 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
@@ -27,6 +27,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
@@ -57,7 +58,7 @@
     public void countOfCountsSingleNC() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileSplit[] splits = new FileSplit[] { new FileSplit(NC1_ID, new File("data/words.txt")) };
+        FileSplit[] splits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File("data/words.txt"))) };
         IFileSplitProvider splitProvider = new ConstantFileSplitProvider(splits);
         RecordDescriptor desc = new RecordDescriptor(
                 new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE });
@@ -136,7 +137,7 @@
     public void countOfCountsMultiNC() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileSplit[] splits = new FileSplit[] { new FileSplit(NC1_ID, new File("data/words.txt")) };
+        FileSplit[] splits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File("data/words.txt"))) };
         IFileSplitProvider splitProvider = new ConstantFileSplitProvider(splits);
         RecordDescriptor desc = new RecordDescriptor(
                 new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE });
@@ -217,7 +218,7 @@
     public void countOfCountsExternalSortMultiNC() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileSplit[] splits = new FileSplit[] { new FileSplit(NC1_ID, new File("data/words.txt")) };
+        FileSplit[] splits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File("data/words.txt"))) };
         IFileSplitProvider splitProvider = new ConstantFileSplitProvider(splits);
         RecordDescriptor desc = new RecordDescriptor(
                 new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE });
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
index 4bb5c3e..5ab05cc 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
@@ -26,6 +26,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.data.hash.IntegerBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
@@ -50,7 +51,8 @@
         JobSpecification spec = new JobSpecification();
 
         IFileSplitProvider splitProvider = new ConstantFileSplitProvider(new FileSplit[] {
-                new FileSplit(NC2_ID, new File("data/words.txt")), new FileSplit(NC1_ID, new File("data/words.txt")) });
+                new FileSplit(NC2_ID, new FileReference(new File("data/words.txt"))),
+                new FileSplit(NC1_ID, new FileReference(new File("data/words.txt"))) });
 
         RecordDescriptor desc = new RecordDescriptor(
                 new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE });
@@ -80,7 +82,8 @@
     public void scanPrint02() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC2_ID, new File("data/tpch0.001/orders.tbl")) };
+        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC2_ID, new FileReference(new File(
+                "data/tpch0.001/orders.tbl"))) };
         IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(ordersSplits);
         RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
@@ -117,7 +120,8 @@
     public void scanPrint03() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC2_ID, new File("data/tpch0.001/orders.tbl")) };
+        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC2_ID, new FileReference(new File(
+                "data/tpch0.001/orders.tbl"))) };
         IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(ordersSplits);
         RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
index feb243e..f2b63ae 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
@@ -26,6 +26,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
@@ -50,8 +51,8 @@
         JobSpecification spec = new JobSpecification();
 
         FileSplit[] ordersSplits = new FileSplit[] {
-                new FileSplit(NC1_ID, new File("data/tpch0.001/orders-part1.tbl")),
-                new FileSplit(NC2_ID, new File("data/tpch0.001/orders-part2.tbl")) };
+                new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/orders-part1.tbl"))),
+                new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/orders-part2.tbl"))) };
         IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
         RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
@@ -96,8 +97,8 @@
         JobSpecification spec = new JobSpecification();
 
         FileSplit[] ordersSplits = new FileSplit[] {
-                new FileSplit(NC1_ID, new File("data/tpch0.001/orders-part1.tbl")),
-                new FileSplit(NC2_ID, new File("data/tpch0.001/orders-part2.tbl")) };
+                new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/orders-part1.tbl"))),
+                new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/orders-part2.tbl"))) };
         IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
         RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
index 5ebbf62..452b840 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
@@ -29,6 +29,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
@@ -62,7 +63,8 @@
     public void customerOrderCIDJoin() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileSplit[] custSplits = new FileSplit[] { new FileSplit(NC1_ID, new File("data/tpch0.001/customer.tbl")) };
+        FileSplit[] custSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
+                "data/tpch0.001/customer.tbl"))) };
         IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(custSplits);
         RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
@@ -70,7 +72,8 @@
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
-        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC2_ID, new File("data/tpch0.001/orders.tbl")) };
+        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC2_ID, new FileReference(new File(
+                "data/tpch0.001/orders.tbl"))) };
         IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(ordersSplits);
         RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
@@ -140,7 +143,8 @@
     public void customerOrderCIDGraceJoin() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileSplit[] custSplits = new FileSplit[] { new FileSplit(NC1_ID, new File("data/tpch0.001/customer.tbl")) };
+        FileSplit[] custSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
+                "data/tpch0.001/customer.tbl"))) };
         IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(custSplits);
         RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
@@ -148,7 +152,8 @@
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
-        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC2_ID, new File("data/tpch0.001/orders.tbl")) };
+        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC2_ID, new FileReference(new File(
+                "data/tpch0.001/orders.tbl"))) };
         IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(ordersSplits);
         RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
@@ -219,7 +224,8 @@
     public void customerOrderCIDHybridHashJoin() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileSplit[] custSplits = new FileSplit[] { new FileSplit(NC1_ID, new File("data/tpch0.001/customer.tbl")) };
+        FileSplit[] custSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
+                "data/tpch0.001/customer.tbl"))) };
         IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(custSplits);
         RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
@@ -227,7 +233,8 @@
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
-        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC2_ID, new File("data/tpch0.001/orders.tbl")) };
+        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC2_ID, new FileReference(new File(
+                "data/tpch0.001/orders.tbl"))) };
         IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(ordersSplits);
         RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
@@ -299,8 +306,8 @@
         JobSpecification spec = new JobSpecification();
 
         FileSplit[] custSplits = new FileSplit[] {
-                new FileSplit(NC1_ID, new File("data/tpch0.001/customer-part1.tbl")),
-                new FileSplit(NC2_ID, new File("data/tpch0.001/customer-part2.tbl")) };
+                new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/customer-part1.tbl"))),
+                new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/customer-part2.tbl"))) };
         IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(custSplits);
         RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
@@ -309,8 +316,8 @@
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
         FileSplit[] ordersSplits = new FileSplit[] {
-                new FileSplit(NC1_ID, new File("data/tpch0.001/orders-part1.tbl")),
-                new FileSplit(NC2_ID, new File("data/tpch0.001/orders-part2.tbl")) };
+                new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/orders-part1.tbl"))),
+                new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/orders-part2.tbl"))) };
         IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(ordersSplits);
         RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
@@ -385,8 +392,8 @@
         JobSpecification spec = new JobSpecification();
 
         FileSplit[] custSplits = new FileSplit[] {
-                new FileSplit(NC1_ID, new File("data/tpch0.001/customer-part1.tbl")),
-                new FileSplit(NC2_ID, new File("data/tpch0.001/customer-part2.tbl")) };
+                new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/customer-part1.tbl"))),
+                new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/customer-part2.tbl"))) };
         IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(custSplits);
         RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
@@ -395,8 +402,8 @@
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
         FileSplit[] ordersSplits = new FileSplit[] {
-                new FileSplit(NC1_ID, new File("data/tpch0.001/orders-part1.tbl")),
-                new FileSplit(NC2_ID, new File("data/tpch0.001/orders-part2.tbl")) };
+                new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/orders-part1.tbl"))),
+                new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/orders-part2.tbl"))) };
         IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(ordersSplits);
         RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
@@ -472,8 +479,8 @@
         JobSpecification spec = new JobSpecification();
 
         FileSplit[] custSplits = new FileSplit[] {
-                new FileSplit(NC1_ID, new File("data/tpch0.001/customer-part1.tbl")),
-                new FileSplit(NC2_ID, new File("data/tpch0.001/customer-part2.tbl")) };
+                new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/customer-part1.tbl"))),
+                new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/customer-part2.tbl"))) };
         IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(custSplits);
         RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
@@ -482,8 +489,8 @@
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
         FileSplit[] ordersSplits = new FileSplit[] {
-                new FileSplit(NC1_ID, new File("data/tpch0.001/orders-part1.tbl")),
-                new FileSplit(NC2_ID, new File("data/tpch0.001/orders-part2.tbl")) };
+                new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/orders-part1.tbl"))),
+                new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/orders-part2.tbl"))) };
         IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(ordersSplits);
         RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
@@ -559,8 +566,8 @@
         JobSpecification spec = new JobSpecification();
 
         FileSplit[] custSplits = new FileSplit[] {
-                new FileSplit(NC1_ID, new File("data/tpch0.001/customer-part1.tbl")),
-                new FileSplit(NC2_ID, new File("data/tpch0.001/customer-part2.tbl")) };
+                new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/customer-part1.tbl"))),
+                new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/customer-part2.tbl"))) };
         IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(custSplits);
         RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
@@ -569,8 +576,8 @@
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
         FileSplit[] ordersSplits = new FileSplit[] {
-                new FileSplit(NC1_ID, new File("data/tpch0.001/orders-part1.tbl")),
-                new FileSplit(NC2_ID, new File("data/tpch0.001/orders-part2.tbl")) };
+                new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/orders-part1.tbl"))),
+                new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/orders-part2.tbl"))) };
         IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(ordersSplits);
         RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
@@ -643,8 +650,8 @@
         JobSpecification spec = new JobSpecification();
 
         FileSplit[] custSplits = new FileSplit[] {
-                new FileSplit(NC1_ID, new File("data/tpch0.001/customer-part1.tbl")),
-                new FileSplit(NC2_ID, new File("data/tpch0.001/customer-part2.tbl")) };
+                new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/customer-part1.tbl"))),
+                new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/customer-part2.tbl"))) };
         IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(custSplits);
         RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
@@ -653,8 +660,8 @@
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
         FileSplit[] ordersSplits = new FileSplit[] {
-                new FileSplit(NC1_ID, new File("data/tpch0.001/orders-part1.tbl")),
-                new FileSplit(NC2_ID, new File("data/tpch0.001/orders-part2.tbl")) };
+                new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/orders-part1.tbl"))),
+                new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/orders-part2.tbl"))) };
         IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(ordersSplits);
         RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java
index e7ac2d5..bdf8ebb 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java
@@ -10,6 +10,7 @@
 import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
@@ -29,39 +30,45 @@
 import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
 
 public class InvertedIndexOperatorsTest extends AbstractIntegrationTest {
-	
-	@Test
-	public void tokenizerTest() throws Exception {	
-		JobSpecification spec = new JobSpecification();
-		
-        FileSplit[] dblpTitleFileSplits = new FileSplit[] {
-                new FileSplit(NC1_ID, new File("data/cleanednumbereddblptitles.txt")) };
+
+    @Test
+    public void tokenizerTest() throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        FileSplit[] dblpTitleFileSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
+                "data/cleanednumbereddblptitles.txt"))) };
         IFileSplitProvider dblpTitleSplitProvider = new ConstantFileSplitProvider(dblpTitleFileSplits);
         RecordDescriptor dblpTitleRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE});
-        
+                IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
+
         FileScanOperatorDescriptor dblpTitleScanner = new FileScanOperatorDescriptor(spec, dblpTitleSplitProvider,
-                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { IntegerParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), dblpTitleRecDesc);
-        PartitionConstraint ordersPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
+                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { IntegerParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE }, '|'), dblpTitleRecDesc);
+        PartitionConstraint ordersPartitionConstraint = new ExplicitPartitionConstraint(
+                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
         dblpTitleScanner.setPartitionConstraint(ordersPartitionConstraint);
 
-        RecordDescriptor tokenizerRecDesc = new RecordDescriptor(new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE } );                
+        RecordDescriptor tokenizerRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
         IBinaryTokenizerFactory tokenizerFactory = new DelimitedUTF8StringBinaryTokenizerFactory(' ');
         int[] tokenFields = { 1 };
         int[] projFields = { 0 };
-        BinaryTokenizerOperatorDescriptor binaryTokenizer = new BinaryTokenizerOperatorDescriptor(spec, tokenizerRecDesc, tokenizerFactory, tokenFields, projFields);	
-        PartitionConstraint tokenizerPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        binaryTokenizer.setPartitionConstraint(tokenizerPartitionConstraint);                        
-        
+        BinaryTokenizerOperatorDescriptor binaryTokenizer = new BinaryTokenizerOperatorDescriptor(spec,
+                tokenizerRecDesc, tokenizerFactory, tokenFields, projFields);
+        PartitionConstraint tokenizerPartitionConstraint = new ExplicitPartitionConstraint(
+                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
+        binaryTokenizer.setPartitionConstraint(tokenizerPartitionConstraint);
+
         PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
-		PartitionConstraint printerPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        printer.setPartitionConstraint(printerPartitionConstraint);				
-        
+        PartitionConstraint printerPartitionConstraint = new ExplicitPartitionConstraint(
+                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
+        printer.setPartitionConstraint(printerPartitionConstraint);
+
         spec.connect(new OneToOneConnectorDescriptor(spec), dblpTitleScanner, 0, binaryTokenizer, 0);
-        
+
         spec.connect(new OneToOneConnectorDescriptor(spec), binaryTokenizer, 0, printer, 0);
-                     
+
         spec.addRoot(printer);
-        runTest(spec);        
-	}	
-}
+        runTest(spec);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java b/hyracks/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
index f69b9fc..84cbd9a 100644
--- a/hyracks/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
+++ b/hyracks/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
@@ -32,6 +32,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
@@ -109,7 +110,7 @@
             if (idx < 0) {
                 throw new IllegalArgumentException("File split " + s + " not well formed");
             }
-            fSplits[i] = new FileSplit(s.substring(0, idx), new File(s.substring(idx + 1)));
+            fSplits[i] = new FileSplit(s.substring(0, idx), new FileReference(new File(s.substring(idx + 1))));
         }
         return fSplits;
     }
diff --git a/hyracks/hyracks-examples/text-example/texthelper/src/main/java/edu/uci/ics/hyracks/examples/text/WordTupleParserFactory.java b/hyracks/hyracks-examples/text-example/texthelper/src/main/java/edu/uci/ics/hyracks/examples/text/WordTupleParserFactory.java
index 9825fdf..65b9011 100644
--- a/hyracks/hyracks-examples/text-example/texthelper/src/main/java/edu/uci/ics/hyracks/examples/text/WordTupleParserFactory.java
+++ b/hyracks/hyracks-examples/text-example/texthelper/src/main/java/edu/uci/ics/hyracks/examples/text/WordTupleParserFactory.java
@@ -9,7 +9,7 @@
 import java.util.Arrays;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
@@ -23,13 +23,13 @@
     private static final long serialVersionUID = 1L;
 
     @Override
-    public ITupleParser createTupleParser(final IHyracksContext ctx) {
+    public ITupleParser createTupleParser(final IHyracksStageletContext ctx) {
         return new ITupleParser() {
             @Override
             public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
                 try {
-                    ByteBuffer frame = ctx.getResourceManager().allocateFrame();
-                    FrameTupleAppender appender = new FrameTupleAppender(ctx);
+                    ByteBuffer frame = ctx.allocateFrame();
+                    FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
                     appender.reset(frame, true);
                     ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
                     DataOutput dos = tb.getDataOutput();
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
index 5bc5479..e26d2a1 100644
--- a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
@@ -16,6 +16,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
@@ -159,7 +160,7 @@
     private static FileSplit[] createOrdersFileSplits() {
         FileSplit fss[] = new FileSplit[2];
         for (int i = 0; i < fss.length; ++i) {
-            fss[i] = new FileSplit("foo", new File("data/tpch0.001/orders-part" + (i + 1) + ".tbl"));
+            fss[i] = new FileSplit("foo", new FileReference(new File("data/tpch0.001/orders-part" + (i + 1) + ".tbl")));
         }
         return fss;
     }
@@ -167,7 +168,8 @@
     private static FileSplit[] createCustomerFileSplits() {
         FileSplit fss[] = new FileSplit[2];
         for (int i = 0; i < fss.length; ++i) {
-            fss[i] = new FileSplit("foo", new File("data/tpch0.001/customer-part" + (i + 1) + ".tbl"));
+            fss[i] = new FileSplit("foo",
+                    new FileReference(new File("data/tpch0.001/customer-part" + (i + 1) + ".tbl")));
         }
         return fss;
     }
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/DummySMI.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/DummySMI.java
deleted file mode 100644
index 9baafcc..0000000
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/DummySMI.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed 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 from
- * 
- *     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 edu.uci.ics.hyracks.storage.am.btree.api;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-import edu.uci.ics.hyracks.storage.common.buffercache.BufferCache;
-import edu.uci.ics.hyracks.storage.common.buffercache.ClockPageReplacementStrategy;
-import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
-import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-import edu.uci.ics.hyracks.storage.common.buffercache.ICacheMemoryAllocator;
-import edu.uci.ics.hyracks.storage.common.buffercache.IPageReplacementStrategy;
-import edu.uci.ics.hyracks.storage.common.file.IFileMapManager;
-import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
-
-public class DummySMI implements IStorageManagerInterface {
-
-    private static final long serialVersionUID = 1L;
-
-    private transient IBufferCache bufferCache = null;
-    private transient IFileMapManager fmManager;
-    private int PAGE_SIZE = 8192;
-    private int NUM_PAGES = 40;
-
-    public DummySMI() {
-    }
-
-    public DummySMI(int pageSize, int numPages) {
-        PAGE_SIZE = pageSize;
-        NUM_PAGES = numPages;
-    }
-
-    @Override
-    public synchronized IBufferCache getBufferCache() {
-        if (bufferCache == null) {
-            ICacheMemoryAllocator allocator = new HeapBufferAllocator();
-            IPageReplacementStrategy prs = new ClockPageReplacementStrategy();
-            fmManager = new IFileMapManager() {
-                private Map<Integer, String> id2nameMap = new HashMap<Integer, String>();
-                private Map<String, Integer> name2IdMap = new HashMap<String, Integer>();
-                private int idCounter = 0;
-
-                @Override
-                public String lookupFileName(int fileId) throws HyracksDataException {
-                    String fName = id2nameMap.get(fileId);
-                    if (fName == null) {
-                        throw new HyracksDataException("No mapping found for id: " + fileId);
-                    }
-                    return fName;
-                }
-
-                @Override
-                public int lookupFileId(String fileName) throws HyracksDataException {
-                    Integer fileId = name2IdMap.get(fileName);
-                    if (fileId == null) {
-                        throw new HyracksDataException("No mapping found for name: " + fileName);
-                    }
-                    return fileId;
-                }
-
-                @Override
-                public boolean isMapped(String fileName) {
-                    return name2IdMap.containsKey(fileName);
-                }
-
-                @Override
-                public boolean isMapped(int fileId) {
-                    return id2nameMap.containsKey(fileId);
-                }
-
-                @Override
-                public void unregisterFile(int fileId) throws HyracksDataException {
-                    String fileName = id2nameMap.remove(fileId);
-                    name2IdMap.remove(fileName);
-                }
-
-                @Override
-                public void registerFile(String fileName) throws HyracksDataException {
-                    Integer fileId = idCounter++;
-                    id2nameMap.put(fileId, fileName);
-                    name2IdMap.put(fileName, fileId);
-                }
-            };
-            bufferCache = new BufferCache(allocator, prs, fmManager, PAGE_SIZE, NUM_PAGES);
-        }
-
-        return bufferCache;
-    }
-
-    public IFileMapProvider getFileMapProvider() {
-        return fmManager;
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeBulkLoadOperatorDescriptor.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeBulkLoadOperatorDescriptor.java
index 6b64b03..a4c4b61 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeBulkLoadOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeBulkLoadOperatorDescriptor.java
@@ -15,7 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.btree.dataflow;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -45,7 +45,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new BTreeBulkLoadOperatorNodePushable(this, ctx, partition, fieldPermutation, fillFactor,
                 recordDescProvider);
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeBulkLoadOperatorNodePushable.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeBulkLoadOperatorNodePushable.java
index 1a868e7..1dd69ba 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeBulkLoadOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeBulkLoadOperatorNodePushable.java
@@ -16,7 +16,7 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -36,7 +36,7 @@
 
     private PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
 
-    public BTreeBulkLoadOperatorNodePushable(AbstractBTreeOperatorDescriptor opDesc, IHyracksContext ctx,
+    public BTreeBulkLoadOperatorNodePushable(AbstractBTreeOperatorDescriptor opDesc, IHyracksStageletContext ctx,
             int partition, int[] fieldPermutation, float fillFactor, IRecordDescriptorProvider recordDescProvider) {
         btreeOpHelper = new BTreeOpHelper(opDesc, ctx, partition, BTreeOpHelper.BTreeMode.CREATE_BTREE);
         this.fillFactor = fillFactor;
@@ -48,7 +48,7 @@
     public void open() throws HyracksDataException {
         AbstractBTreeOperatorDescriptor opDesc = btreeOpHelper.getOperatorDescriptor();
         RecordDescriptor recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
-        accessor = new FrameTupleAccessor(btreeOpHelper.getHyracksContext(), recDesc);
+        accessor = new FrameTupleAccessor(btreeOpHelper.getHyracksStageletContext().getFrameSize(), recDesc);
         IBTreeMetaDataFrame metaFrame = new MetaDataFrame();
         btreeOpHelper.init();
         btreeOpHelper.getBTree().open(btreeOpHelper.getBTreeFileId());
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDiskOrderScanOperatorDescriptor.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDiskOrderScanOperatorDescriptor.java
index ddc7c39..17fbf97 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDiskOrderScanOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDiskOrderScanOperatorDescriptor.java
@@ -15,7 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.btree.dataflow;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
@@ -40,8 +40,8 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new BTreeDiskOrderScanOperatorNodePushable(this, ctx, partition);
     }
-}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDiskOrderScanOperatorNodePushable.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDiskOrderScanOperatorNodePushable.java
index c3a0df2..fe71c33 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDiskOrderScanOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDiskOrderScanOperatorNodePushable.java
@@ -17,7 +17,7 @@
 import java.io.DataOutput;
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
@@ -33,7 +33,7 @@
 public class BTreeDiskOrderScanOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
     private final BTreeOpHelper btreeOpHelper;
 
-    public BTreeDiskOrderScanOperatorNodePushable(AbstractBTreeOperatorDescriptor opDesc, IHyracksContext ctx,
+    public BTreeDiskOrderScanOperatorNodePushable(AbstractBTreeOperatorDescriptor opDesc, IHyracksStageletContext ctx,
             int partition) {
         btreeOpHelper = new BTreeOpHelper(opDesc, ctx, partition, BTreeOpHelper.BTreeMode.OPEN_BTREE);
     }
@@ -49,8 +49,8 @@
         btreeOpHelper.getBTree().diskOrderScan(cursor, cursorFrame, metaFrame);
 
         MultiComparator cmp = btreeOpHelper.getBTree().getMultiComparator();
-        ByteBuffer frame = btreeOpHelper.getHyracksContext().getResourceManager().allocateFrame();
-        FrameTupleAppender appender = new FrameTupleAppender(btreeOpHelper.getHyracksContext());
+        ByteBuffer frame = btreeOpHelper.getHyracksStageletContext().allocateFrame();
+        FrameTupleAppender appender = new FrameTupleAppender(btreeOpHelper.getHyracksStageletContext().getFrameSize());
         appender.reset(frame, true);
         ArrayTupleBuilder tb = new ArrayTupleBuilder(cmp.getFieldCount());
         DataOutput dos = tb.getDataOutput();
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDropOperatorDescriptor.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDropOperatorDescriptor.java
index 5a04d79..970d44a 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDropOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDropOperatorDescriptor.java
@@ -15,7 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.btree.dataflow;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
@@ -41,8 +41,8 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
-        return new BTreeDropOperatorNodePushable(storageManager, btreeRegistryProvider, fileSplitProvider, partition);
+        return new BTreeDropOperatorNodePushable(ctx, storageManager, btreeRegistryProvider, fileSplitProvider, partition);
     }
 }
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDropOperatorNodePushable.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDropOperatorNodePushable.java
index 3745fad..23a6601 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDropOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDropOperatorNodePushable.java
@@ -15,11 +15,11 @@
 
 package edu.uci.ics.hyracks.storage.am.btree.dataflow;
 
-import java.io.File;
-
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorNodePushable;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
@@ -27,14 +27,15 @@
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 
 public class BTreeDropOperatorNodePushable extends AbstractOperatorNodePushable {
-
+    private final IHyracksStageletContext ctx;
     private IBTreeRegistryProvider btreeRegistryProvider;
     private IStorageManagerInterface storageManager;
     private IFileSplitProvider fileSplitProvider;
     private int partition;
 
-    public BTreeDropOperatorNodePushable(IStorageManagerInterface storageManager,
+    public BTreeDropOperatorNodePushable(IHyracksStageletContext ctx, IStorageManagerInterface storageManager,
             IBTreeRegistryProvider btreeRegistryProvider, IFileSplitProvider fileSplitProvider, int partition) {
+        this.ctx = ctx;
         this.storageManager = storageManager;
         this.btreeRegistryProvider = btreeRegistryProvider;
         this.fileSplitProvider = fileSplitProvider;
@@ -58,19 +59,18 @@
     @Override
     public void initialize() throws HyracksDataException {
 
-        BTreeRegistry btreeRegistry = btreeRegistryProvider.getBTreeRegistry();
-        IBufferCache bufferCache = storageManager.getBufferCache();
-        IFileMapProvider fileMapProvider = storageManager.getFileMapProvider();
+        BTreeRegistry btreeRegistry = btreeRegistryProvider.getBTreeRegistry(ctx);
+        IBufferCache bufferCache = storageManager.getBufferCache(ctx);
+        IFileMapProvider fileMapProvider = storageManager.getFileMapProvider(ctx);
 
-        File f = fileSplitProvider.getFileSplits()[partition].getLocalFile();
-        String fileName = f.getAbsolutePath();
+        FileReference f = fileSplitProvider.getFileSplits()[partition].getLocalFile();
 
-        boolean fileIsMapped = fileMapProvider.isMapped(fileName);
+        boolean fileIsMapped = fileMapProvider.isMapped(f);
         if (!fileIsMapped) {
-            throw new HyracksDataException("Cannot drop B-Tree with name " + fileName + ". No file mapping exists.");
+            throw new HyracksDataException("Cannot drop B-Tree with name " + f.toString() + ". No file mapping exists.");
         }
 
-        int btreeFileId = fileMapProvider.lookupFileId(fileName);
+        int btreeFileId = fileMapProvider.lookupFileId(f);
 
         // unregister btree instance
         btreeRegistry.lock();
@@ -82,15 +82,9 @@
 
         // remove name to id mapping
         bufferCache.deleteFile(btreeFileId);
-
-        // TODO: should this be handled through the BufferCache or
-        // FileMapProvider?
-        if (f.exists()) {
-            f.delete();
-        }
     }
 
     @Override
     public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
     }
-}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeFileEnlistmentOperatorDescriptor.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeFileEnlistmentOperatorDescriptor.java
index d66e7c0..c105274 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeFileEnlistmentOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeFileEnlistmentOperatorDescriptor.java
@@ -15,7 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.btree.dataflow;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -47,7 +47,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int partitions) throws HyracksDataException {
         return new BTreeFileEnlistmentOperatorNodePushable(this, ctx, partition);
     }
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeFileEnlistmentOperatorNodePushable.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeFileEnlistmentOperatorNodePushable.java
index 3e48a01..7557573 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeFileEnlistmentOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeFileEnlistmentOperatorNodePushable.java
@@ -16,7 +16,7 @@
 package edu.uci.ics.hyracks.storage.am.btree.dataflow;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorNodePushable;
@@ -25,7 +25,7 @@
 
     private final BTreeOpHelper btreeOpHelper;
 
-    public BTreeFileEnlistmentOperatorNodePushable(AbstractBTreeOperatorDescriptor opDesc, IHyracksContext ctx,
+    public BTreeFileEnlistmentOperatorNodePushable(AbstractBTreeOperatorDescriptor opDesc, IHyracksStageletContext ctx,
             int partition) {
         btreeOpHelper = new BTreeOpHelper(opDesc, ctx, partition, BTreeOpHelper.BTreeMode.ENLIST_BTREE);
     }
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeInsertUpdateDeleteOperatorDescriptor.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeInsertUpdateDeleteOperatorDescriptor.java
index 71e1a16..7d50fff 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeInsertUpdateDeleteOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeInsertUpdateDeleteOperatorDescriptor.java
@@ -15,7 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.btree.dataflow;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -49,7 +49,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new BTreeInsertUpdateDeleteOperatorNodePushable(this, ctx, partition, fieldPermutation,
                 recordDescProvider, op);
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeInsertUpdateDeleteOperatorNodePushable.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeInsertUpdateDeleteOperatorNodePushable.java
index a4296cb..e6ba012 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeInsertUpdateDeleteOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeInsertUpdateDeleteOperatorNodePushable.java
@@ -16,7 +16,7 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -37,8 +37,9 @@
     private ByteBuffer writeBuffer;
     private BTreeOpContext opCtx;
 
-    public BTreeInsertUpdateDeleteOperatorNodePushable(AbstractBTreeOperatorDescriptor opDesc, IHyracksContext ctx,
-            int partition, int[] fieldPermutation, IRecordDescriptorProvider recordDescProvider, BTreeOp op) {
+    public BTreeInsertUpdateDeleteOperatorNodePushable(AbstractBTreeOperatorDescriptor opDesc,
+            IHyracksStageletContext ctx, int partition, int[] fieldPermutation,
+            IRecordDescriptorProvider recordDescProvider, BTreeOp op) {
         btreeOpHelper = new BTreeOpHelper(opDesc, ctx, partition, BTreeOpHelper.BTreeMode.OPEN_BTREE);
         this.recordDescProvider = recordDescProvider;
         this.op = op;
@@ -49,8 +50,8 @@
     public void open() throws HyracksDataException {
         AbstractBTreeOperatorDescriptor opDesc = btreeOpHelper.getOperatorDescriptor();
         RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
-        accessor = new FrameTupleAccessor(btreeOpHelper.getHyracksContext(), inputRecDesc);
-        writeBuffer = btreeOpHelper.getHyracksContext().getResourceManager().allocateFrame();
+        accessor = new FrameTupleAccessor(btreeOpHelper.getHyracksStageletContext().getFrameSize(), inputRecDesc);
+        writeBuffer = btreeOpHelper.getHyracksStageletContext().allocateFrame();
         btreeOpHelper.init();
         btreeOpHelper.getBTree().open(btreeOpHelper.getBTreeFileId());
         opCtx = btreeOpHelper.getBTree().createOpContext(op, btreeOpHelper.getLeafFrame(),
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelper.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelper.java
index 45d1b13..d01ceee 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelper.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelper.java
@@ -14,11 +14,10 @@
  */
 package edu.uci.ics.hyracks.storage.am.btree.dataflow;
 
-import java.io.File;
-
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
@@ -31,7 +30,9 @@
 final class BTreeOpHelper {
 
     public enum BTreeMode {
-        OPEN_BTREE, CREATE_BTREE, ENLIST_BTREE
+        OPEN_BTREE,
+        CREATE_BTREE,
+        ENLIST_BTREE
     }
 
     private IBTreeInteriorFrame interiorFrame;
@@ -42,11 +43,12 @@
     private int partition;
 
     private AbstractBTreeOperatorDescriptor opDesc;
-    private IHyracksContext ctx;
+    private IHyracksStageletContext ctx;
 
     private BTreeMode mode;
 
-    BTreeOpHelper(AbstractBTreeOperatorDescriptor opDesc, final IHyracksContext ctx, int partition, BTreeMode mode) {
+    BTreeOpHelper(AbstractBTreeOperatorDescriptor opDesc, final IHyracksStageletContext ctx, int partition,
+            BTreeMode mode) {
         this.opDesc = opDesc;
         this.ctx = ctx;
         this.mode = mode;
@@ -54,20 +56,18 @@
     }
 
     void init() throws HyracksDataException {
-
-        IBufferCache bufferCache = opDesc.getStorageManager().getBufferCache();
-        IFileMapProvider fileMapProvider = opDesc.getStorageManager().getFileMapProvider();
+        IBufferCache bufferCache = opDesc.getStorageManager().getBufferCache(ctx);
+        IFileMapProvider fileMapProvider = opDesc.getStorageManager().getFileMapProvider(ctx);
         IFileSplitProvider fileSplitProvider = opDesc.getFileSplitProvider();
 
-        File f = fileSplitProvider.getFileSplits()[partition].getLocalFile();
-        String fileName = f.getAbsolutePath();
-        boolean fileIsMapped = fileMapProvider.isMapped(fileName);
+        FileReference f = fileSplitProvider.getFileSplits()[partition].getLocalFile();
+        boolean fileIsMapped = fileMapProvider.isMapped(f);
 
         switch (mode) {
 
             case OPEN_BTREE: {
                 if (!fileIsMapped) {
-                    bufferCache.createFile(fileName);
+                    bufferCache.createFile(f);
                     // throw new
                     // HyracksDataException("Trying to open btree from unmapped file "
                     // + fileName);
@@ -78,20 +78,20 @@
             case CREATE_BTREE:
             case ENLIST_BTREE: {
                 if (!fileIsMapped) {
-                    bufferCache.createFile(fileName);
+                    bufferCache.createFile(f);
                 }
             }
                 break;
 
         }
 
-        btreeFileId = fileMapProvider.lookupFileId(fileName);
+        btreeFileId = fileMapProvider.lookupFileId(f);
         bufferCache.openFile(btreeFileId);
 
         interiorFrame = opDesc.getInteriorFactory().getFrame();
         leafFrame = opDesc.getLeafFactory().getFrame();
 
-        BTreeRegistry btreeRegistry = opDesc.getBtreeRegistryProvider().getBTreeRegistry();
+        BTreeRegistry btreeRegistry = opDesc.getBtreeRegistryProvider().getBTreeRegistry(ctx);
         btree = btreeRegistry.get(btreeFileId);
         if (btree == null) {
 
@@ -130,7 +130,7 @@
 
     public void deinit() throws HyracksDataException {
         if (btreeFileId != -1) {
-            IBufferCache bufferCache = opDesc.getStorageManager().getBufferCache();
+            IBufferCache bufferCache = opDesc.getStorageManager().getBufferCache(ctx);
             bufferCache.closeFile(btreeFileId);
         }
     }
@@ -139,7 +139,7 @@
         return btree;
     }
 
-    public IHyracksContext getHyracksContext() {
+    public IHyracksStageletContext getHyracksStageletContext() {
         return ctx;
     }
 
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
index dd67762..21f37a3 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
@@ -15,7 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.btree.dataflow;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -55,7 +55,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx, final IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new BTreeSearchOperatorNodePushable(this, ctx, partition, recordDescProvider, isForward, lowKeyFields,
                 highKeyFields, lowKeyInclusive, highKeyInclusive);
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
index d06e2ce..eeee17d 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
@@ -17,7 +17,7 @@
 import java.io.DataOutput;
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -61,9 +61,9 @@
 
     private RecordDescriptor recDesc;
 
-    public BTreeSearchOperatorNodePushable(AbstractBTreeOperatorDescriptor opDesc, IHyracksContext ctx, int partition,
-            IRecordDescriptorProvider recordDescProvider, boolean isForward, int[] lowKeyFields, int[] highKeyFields,
-            boolean lowKeyInclusive, boolean highKeyInclusive) {
+    public BTreeSearchOperatorNodePushable(AbstractBTreeOperatorDescriptor opDesc, IHyracksStageletContext ctx,
+            int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward, int[] lowKeyFields,
+            int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive) {
         btreeOpHelper = new BTreeOpHelper(opDesc, ctx, partition, BTreeOpHelper.BTreeMode.OPEN_BTREE);
         this.isForward = isForward;
         this.lowKeyInclusive = lowKeyInclusive;
@@ -82,7 +82,7 @@
     @Override
     public void open() throws HyracksDataException {
         AbstractBTreeOperatorDescriptor opDesc = btreeOpHelper.getOperatorDescriptor();
-        accessor = new FrameTupleAccessor(btreeOpHelper.getHyracksContext(), recDesc);
+        accessor = new FrameTupleAccessor(btreeOpHelper.getHyracksStageletContext().getFrameSize(), recDesc);
 
         cursorFrame = opDesc.getLeafFactory().getFrame();
         cursor = new RangeSearchCursor(cursorFrame);
@@ -119,16 +119,16 @@
         rangePred = new RangePredicate(isForward, null, null, lowKeyInclusive, highKeyInclusive, lowKeySearchCmp,
                 highKeySearchCmp);
 
-        accessor = new FrameTupleAccessor(btreeOpHelper.getHyracksContext(), recDesc);
+        accessor = new FrameTupleAccessor(btreeOpHelper.getHyracksStageletContext().getFrameSize(), recDesc);
 
-        writeBuffer = btreeOpHelper.getHyracksContext().getResourceManager().allocateFrame();
+        writeBuffer = btreeOpHelper.getHyracksStageletContext().allocateFrame();
         tb = new ArrayTupleBuilder(btree.getMultiComparator().getFieldCount());
         dos = tb.getDataOutput();
-        appender = new FrameTupleAppender(btreeOpHelper.getHyracksContext());
+        appender = new FrameTupleAppender(btreeOpHelper.getHyracksStageletContext().getFrameSize());
         appender.reset(writeBuffer, true);
 
-        opCtx = btree.createOpContext(BTreeOp.BTO_SEARCH, btreeOpHelper.getLeafFrame(), btreeOpHelper
-                .getInteriorFrame(), null);
+        opCtx = btree.createOpContext(BTreeOp.BTO_SEARCH, btreeOpHelper.getLeafFrame(),
+                btreeOpHelper.getInteriorFrame(), null);
     }
 
     private void writeSearchResults() throws Exception {
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/HyracksSimpleStorageManagerInterface.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/HyracksSimpleStorageManagerInterface.java
deleted file mode 100644
index 12538d2..0000000
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/HyracksSimpleStorageManagerInterface.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed 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 from
- * 
- *     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 edu.uci.ics.hyracks.storage.am.btree.dataflow;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-import edu.uci.ics.hyracks.storage.common.buffercache.BufferCache;
-import edu.uci.ics.hyracks.storage.common.buffercache.ClockPageReplacementStrategy;
-import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
-import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-import edu.uci.ics.hyracks.storage.common.buffercache.ICacheMemoryAllocator;
-import edu.uci.ics.hyracks.storage.common.buffercache.IPageReplacementStrategy;
-import edu.uci.ics.hyracks.storage.common.file.IFileMapManager;
-import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
-
-public class HyracksSimpleStorageManagerInterface implements IStorageManagerInterface {
-    private static final long serialVersionUID = 1L;
-
-    private static transient IBufferCache bufferCache = null;
-    private static transient IFileMapManager fmManager;
-    private int PAGE_SIZE = 8192;
-    private int NUM_PAGES = 40;
-
-    public HyracksSimpleStorageManagerInterface() {
-        init();
-    }
-
-    public HyracksSimpleStorageManagerInterface(int pageSize, int numPages) {
-        PAGE_SIZE = pageSize;
-        NUM_PAGES = numPages;
-        init();
-    }
-
-    private void init() {
-        ICacheMemoryAllocator allocator = new HeapBufferAllocator();
-        IPageReplacementStrategy prs = new ClockPageReplacementStrategy();
-        fmManager = new IFileMapManager() {
-            private Map<Integer, String> id2nameMap = new HashMap<Integer, String>();
-            private Map<String, Integer> name2IdMap = new HashMap<String, Integer>();
-            private int idCounter = 0;
-
-            @Override
-            public String lookupFileName(int fileId) throws HyracksDataException {
-                String fName = id2nameMap.get(fileId);
-                if (fName == null) {
-                    throw new HyracksDataException("No mapping found for id: " + fileId);
-                }
-                return fName;
-            }
-
-            @Override
-            public int lookupFileId(String fileName) throws HyracksDataException {
-                Integer fileId = name2IdMap.get(fileName);
-                if (fileId == null) {
-                    throw new HyracksDataException("No mapping found for name: " + fileName);
-                }
-                return fileId;
-            }
-
-            @Override
-            public boolean isMapped(String fileName) {
-                return name2IdMap.containsKey(fileName);
-            }
-
-            @Override
-            public boolean isMapped(int fileId) {
-                return id2nameMap.containsKey(fileId);
-            }
-
-            @Override
-            public void unregisterFile(int fileId) throws HyracksDataException {
-                String fileName = id2nameMap.remove(fileId);
-                name2IdMap.remove(fileName);
-            }
-
-            @Override
-            public void registerFile(String fileName) throws HyracksDataException {
-                Integer fileId = idCounter++;
-                id2nameMap.put(fileId, fileName);
-                name2IdMap.put(fileName, fileId);
-            }
-        };
-        bufferCache = new BufferCache(allocator, prs, fmManager, PAGE_SIZE, NUM_PAGES);
-    }
-
-    public IFileMapProvider getFileMapProvider() {
-        return fmManager;
-    }
-
-    @Override
-    public IBufferCache getBufferCache() {
-        return bufferCache;
-    }
-}
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/IBTreeRegistryProvider.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/IBTreeRegistryProvider.java
index 1f0f1ef..7b66d0a 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/IBTreeRegistryProvider.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/IBTreeRegistryProvider.java
@@ -17,6 +17,8 @@
 
 import java.io.Serializable;
 
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+
 public interface IBTreeRegistryProvider extends Serializable {
-    public BTreeRegistry getBTreeRegistry();
-}
+    public BTreeRegistry getBTreeRegistry(IHyracksStageletContext ctx);
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
index 0c47e12..7a8ee73 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
@@ -33,7 +33,7 @@
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMInteriorFrame;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
-import edu.uci.ics.hyracks.storage.common.file.FileHandle;
+import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
 
 public class BTree {
 
@@ -113,7 +113,7 @@
                 return;
 
             // initialize meta data page
-            ICachedPage metaNode = bufferCache.pin(FileHandle.getDiskPageId(fileId, metaDataPage), false);
+            ICachedPage metaNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, metaDataPage), false);
             pins++;
 
             metaNode.acquireWriteLatch();
@@ -130,7 +130,7 @@
             }
 
             // initialize root page
-            ICachedPage rootNode = bufferCache.pin(FileHandle.getDiskPageId(fileId, rootPage), true);
+            ICachedPage rootNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rootPage), true);
             pins++;
 
             rootNode.acquireWriteLatch();
@@ -161,7 +161,7 @@
     }
 
     private int getFreePage(IBTreeMetaDataFrame metaFrame) throws HyracksDataException {
-        ICachedPage metaNode = bufferCache.pin(FileHandle.getDiskPageId(fileId, metaDataPage), false);
+        ICachedPage metaNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, metaDataPage), false);
         pins++;
 
         metaNode.acquireWriteLatch();
@@ -174,7 +174,7 @@
             if (freePage < 0) { // no free page entry on this page
                 int nextPage = metaFrame.getNextPage();
                 if (nextPage > 0) { // sibling may have free pages
-                    ICachedPage nextNode = bufferCache.pin(FileHandle.getDiskPageId(fileId, nextPage), false);
+                    ICachedPage nextNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, nextPage), false);
                     pins++;
 
                     nextNode.acquireWriteLatch();
@@ -238,7 +238,7 @@
         if (freePage == rootPage)
             return;
 
-        ICachedPage metaNode = bufferCache.pin(FileHandle.getDiskPageId(fileId, metaDataPage), false);
+        ICachedPage metaNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, metaDataPage), false);
         pins++;
 
         metaNode.acquireWriteLatch();
@@ -256,7 +256,7 @@
                     throw new Exception("Inconsistent Meta Page State. It has no space, but it also has no entries.");
                 }
 
-                ICachedPage newNode = bufferCache.pin(FileHandle.getDiskPageId(fileId, newPage), false);
+                ICachedPage newNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, newPage), false);
                 pins++;
 
                 newNode.acquireWriteLatch();
@@ -293,7 +293,7 @@
     }
 
     public int getMaxPage(IBTreeMetaDataFrame metaFrame) throws HyracksDataException {
-        ICachedPage metaNode = bufferCache.pin(FileHandle.getDiskPageId(fileId, metaDataPage), false);
+        ICachedPage metaNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, metaDataPage), false);
         pins++;
 
         metaNode.acquireWriteLatch();
@@ -320,7 +320,7 @@
     public void printTree(int pageId, ICachedPage parent, boolean unpin, IBTreeLeafFrame leafFrame,
             IBTreeInteriorFrame interiorFrame, ISerializerDeserializer[] fields) throws Exception {
 
-        ICachedPage node = bufferCache.pin(FileHandle.getDiskPageId(fileId, pageId), false);
+        ICachedPage node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
         pins++;
         node.acquireReadLatch();
         readLatchesAcquired++;
@@ -379,7 +379,7 @@
         int currentPageId = rootPage + 1;
         int maxPageId = -1;
 
-        ICachedPage metaNode = bufferCache.pin(FileHandle.getDiskPageId(fileId, metaDataPage), false);
+        ICachedPage metaNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, metaDataPage), false);
         pins++;
 
         metaNode.acquireReadLatch();
@@ -396,7 +396,7 @@
             unpins++;
         }
 
-        ICachedPage page = bufferCache.pin(FileHandle.getDiskPageId(fileId, currentPageId), false);
+        ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), false);
         page.acquireReadLatch();
         cursor.setBufferCache(bufferCache);
         cursor.setFileId(fileId);
@@ -439,7 +439,7 @@
         ICachedPage originalPage = ctx.interiorFrame.getPage();
         for (int i = 0; i < ctx.smPages.size(); i++) {
             int pageId = ctx.smPages.get(i);
-            ICachedPage smPage = bufferCache.pin(FileHandle.getDiskPageId(fileId, pageId), false);
+            ICachedPage smPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
             pins++;
             smPage.acquireWriteLatch(); // TODO: would like to set page dirty
             // without latching
@@ -468,13 +468,13 @@
         currentLevel++;
 
         // make sure the root is always at the same level
-        ICachedPage leftNode = bufferCache.pin(FileHandle.getDiskPageId(fileId, ctx.splitKey.getLeftPage()), false);
+        ICachedPage leftNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, ctx.splitKey.getLeftPage()), false);
         pins++;
         leftNode.acquireWriteLatch(); // TODO: think about whether latching is
         // really required
         writeLatchesAcquired++;
         try {
-            ICachedPage rightNode = bufferCache.pin(FileHandle.getDiskPageId(fileId, ctx.splitKey.getRightPage()),
+            ICachedPage rightNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, ctx.splitKey.getRightPage()),
                     false);
             pins++;
             rightNode.acquireWriteLatch(); // TODO: think about whether latching
@@ -482,7 +482,7 @@
             writeLatchesAcquired++;
             try {
                 int newLeftId = getFreePage(ctx.metaFrame);
-                ICachedPage newLeftNode = bufferCache.pin(FileHandle.getDiskPageId(fileId, newLeftId), true);
+                ICachedPage newLeftNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, newLeftId), true);
                 pins++;
                 newLeftNode.acquireWriteLatch(); // TODO: think about whether
                 // latching is really
@@ -610,7 +610,7 @@
                     int rightSiblingPageId = ctx.leafFrame.getNextLeaf();
                     ICachedPage rightSibling = null;
                     if (rightSiblingPageId > 0) {
-                        rightSibling = bufferCache.pin(FileHandle.getDiskPageId(fileId, rightSiblingPageId), false);
+                        rightSibling = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rightSiblingPageId), false);
                         pins++;
                     }
 
@@ -621,7 +621,7 @@
                     try {
 
                         int rightPageId = getFreePage(ctx.metaFrame);
-                        ICachedPage rightNode = bufferCache.pin(FileHandle.getDiskPageId(fileId, rightPageId), true);
+                        ICachedPage rightNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rightPageId), true);
                         pins++;
                         rightNode.acquireWriteLatch();
                         writeLatchesAcquired++;
@@ -709,7 +709,7 @@
             case INSUFFICIENT_SPACE: {
                 splitsByLevel[ctx.interiorFrame.getLevel()]++; // debug
                 int rightPageId = getFreePage(ctx.metaFrame);
-                ICachedPage rightNode = bufferCache.pin(FileHandle.getDiskPageId(fileId, rightPageId), true);
+                ICachedPage rightNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rightPageId), true);
                 pins++;
                 rightNode.acquireWriteLatch();
                 writeLatchesAcquired++;
@@ -792,7 +792,7 @@
 
             // tree is empty, reset level to zero
             if (ctx.splitKey.getBuffer() != null) {
-                ICachedPage rootNode = bufferCache.pin(FileHandle.getDiskPageId(fileId, rootPage), false);
+                ICachedPage rootNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rootPage), false);
                 pins++;
                 rootNode.acquireWriteLatch();
                 writeLatchesAcquired++;
@@ -830,12 +830,12 @@
             int prevLeaf = ctx.leafFrame.getPrevLeaf();
 
             if (prevLeaf > 0)
-                leftNode = bufferCache.pin(FileHandle.getDiskPageId(fileId, prevLeaf), false);
+                leftNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, prevLeaf), false);
 
             try {
 
                 if (nextLeaf > 0)
-                    rightNode = bufferCache.pin(FileHandle.getDiskPageId(fileId, nextLeaf), false);
+                    rightNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, nextLeaf), false);
 
                 try {
                     treeLatch.writeLock().lock();
@@ -976,7 +976,7 @@
     }
 
     private boolean isConsistent(int pageId, BTreeOpContext ctx) throws Exception {
-        ICachedPage node = bufferCache.pin(FileHandle.getDiskPageId(fileId, pageId), false);
+        ICachedPage node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
         pins++;
         node.acquireReadLatch();
         readLatchesAcquired++;
@@ -994,7 +994,7 @@
     }
 
     private void performOp(int pageId, ICachedPage parent, BTreeOpContext ctx) throws Exception {
-        ICachedPage node = bufferCache.pin(FileHandle.getDiskPageId(fileId, pageId), false);
+        ICachedPage node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
         pins++;
 
         ctx.interiorFrame.setPage(node);
@@ -1055,7 +1055,7 @@
 
                             case BTO_INSERT: {
                                 if (ctx.splitKey.getBuffer() != null) {
-                                    node = bufferCache.pin(FileHandle.getDiskPageId(fileId, pageId), false);
+                                    node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
                                     pins++;
                                     node.acquireWriteLatch();
                                     writeLatchesAcquired++;
@@ -1075,7 +1075,7 @@
 
                             case BTO_DELETE: {
                                 if (ctx.splitKey.getBuffer() != null) {
-                                    node = bufferCache.pin(FileHandle.getDiskPageId(fileId, pageId), false);
+                                    node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
                                     pins++;
                                     node.acquireWriteLatch();
                                     writeLatchesAcquired++;
@@ -1194,7 +1194,7 @@
 
             NodeFrontier leafFrontier = new NodeFrontier(leafFrame.createTupleReference());
             leafFrontier.pageId = getFreePage(metaFrame);
-            leafFrontier.page = bufferCache.pin(FileHandle.getDiskPageId(fileId, leafFrontier.pageId), bulkNewPage);
+            leafFrontier.page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId), bulkNewPage);
             leafFrontier.page.acquireWriteLatch();
 
             interiorFrame.setPage(leafFrontier.page);
@@ -1217,7 +1217,7 @@
         private void addLevel() throws HyracksDataException {
             NodeFrontier frontier = new NodeFrontier(tupleWriter.createTupleReference());
             frontier.pageId = getFreePage(metaFrame);
-            frontier.page = bufferCache.pin(FileHandle.getDiskPageId(fileId, frontier.pageId), bulkNewPage);
+            frontier.page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, frontier.pageId), bulkNewPage);
             frontier.page.acquireWriteLatch();
             frontier.lastTuple.setFieldCount(cmp.getKeyFieldCount());
             interiorFrame.setPage(frontier.page);
@@ -1263,7 +1263,7 @@
             ctx.splitKey.setRightPage(frontier.pageId);
             propagateBulk(ctx, level + 1);
 
-            frontier.page = bufferCache.pin(FileHandle.getDiskPageId(fileId, frontier.pageId), bulkNewPage);
+            frontier.page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, frontier.pageId), bulkNewPage);
             frontier.page.acquireWriteLatch();
             ctx.interiorFrame.setPage(frontier.page);
             ctx.interiorFrame.initBuffer((byte) level);
@@ -1322,7 +1322,7 @@
             ctx.splitKey.setRightPage(leafFrontier.pageId);
             propagateBulk(ctx, 1);
 
-            leafFrontier.page = bufferCache.pin(FileHandle.getDiskPageId(fileId, leafFrontier.pageId), bulkNewPage);
+            leafFrontier.page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId), bulkNewPage);
             leafFrontier.page.acquireWriteLatch();
             leafFrame.setPage(leafFrontier.page);
             leafFrame.initBuffer((byte) 0);
@@ -1342,7 +1342,7 @@
 
     public void endBulkLoad(BulkLoadContext ctx) throws HyracksDataException {
         // copy root
-        ICachedPage rootNode = bufferCache.pin(FileHandle.getDiskPageId(fileId, rootPage), bulkNewPage);
+        ICachedPage rootNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rootPage), bulkNewPage);
         rootNode.acquireWriteLatch();
         try {
             ICachedPage toBeRoot = ctx.nodeFrontiers.get(ctx.nodeFrontiers.size() - 1).page;
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/DiskOrderScanCursor.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/DiskOrderScanCursor.java
index e213023..29086ba 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/DiskOrderScanCursor.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/DiskOrderScanCursor.java
@@ -22,7 +22,7 @@
 import edu.uci.ics.hyracks.storage.am.btree.api.ISearchPredicate;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
-import edu.uci.ics.hyracks.storage.common.file.FileHandle;
+import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
 
 public class DiskOrderScanCursor implements IBTreeCursor {
 
@@ -66,7 +66,7 @@
         while ((frame.getLevel() != 0 || skipCurrent) && (currentPageId <= maxPageId) || (frame.getTupleCount() == 0)) {
             currentPageId++;
 
-            ICachedPage nextPage = bufferCache.pin(FileHandle.getDiskPageId(fileId, currentPageId), false);
+            ICachedPage nextPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), false);
             nextPage.acquireReadLatch();
 
             page.releaseReadLatch();
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/RangeSearchCursor.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/RangeSearchCursor.java
index 3d3704c..433cc6a 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/RangeSearchCursor.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/RangeSearchCursor.java
@@ -23,7 +23,7 @@
 import edu.uci.ics.hyracks.storage.am.btree.api.ISearchPredicate;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
-import edu.uci.ics.hyracks.storage.common.file.FileHandle;
+import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
 
 public class RangeSearchCursor implements IBTreeCursor {
 
@@ -74,7 +74,7 @@
     }
 
     private void fetchNextLeafPage(int nextLeafPage) throws HyracksDataException {
-        ICachedPage nextLeaf = bufferCache.pin(FileHandle.getDiskPageId(fileId, nextLeafPage), false);
+        ICachedPage nextLeaf = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, nextLeafPage), false);
         nextLeaf.acquireReadLatch();
 
         page.releaseReadLatch();
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
index d438685..059a1e2 100644
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
@@ -15,7 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.invertedindex.dataflow;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -47,7 +47,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
         return new BinaryTokenizerOperatorNodePushable(ctx, recordDescProvider.getInputRecordDescriptor(odId, 0),
                 recordDescriptors[0], tokenizerFactory.createBinaryTokenizer(), tokenFields, projFields);
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
index ffa8524..9dac535 100644
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
@@ -19,7 +19,7 @@
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -31,7 +31,7 @@
 
 public class BinaryTokenizerOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
 
-    private final IHyracksContext ctx;
+    private final IHyracksStageletContext ctx;
     private final IBinaryTokenizer tokenizer;
     private final int[] tokenFields;
     private final int[] projFields;
@@ -44,7 +44,7 @@
     private FrameTupleAppender appender;
     private ByteBuffer writeBuffer;
 
-    public BinaryTokenizerOperatorNodePushable(IHyracksContext ctx, RecordDescriptor inputRecDesc,
+    public BinaryTokenizerOperatorNodePushable(IHyracksStageletContext ctx, RecordDescriptor inputRecDesc,
             RecordDescriptor outputRecDesc, IBinaryTokenizer tokenizer, int[] tokenFields, int[] projFields) {
         this.ctx = ctx;
         this.tokenizer = tokenizer;
@@ -56,11 +56,11 @@
 
     @Override
     public void open() throws HyracksDataException {
-        accessor = new FrameTupleAccessor(ctx, inputRecDesc);
-        writeBuffer = ctx.getResourceManager().allocateFrame();
+        accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRecDesc);
+        writeBuffer = ctx.allocateFrame();
         builder = new ArrayTupleBuilder(outputRecDesc.getFields().length);
         builderDos = builder.getDataOutput();
-        appender = new FrameTupleAppender(ctx);
+        appender = new FrameTupleAppender(ctx.getFrameSize());
         appender.reset(writeBuffer, true);
     }
 
@@ -73,9 +73,11 @@
 
             for (int j = 0; j < tokenFields.length; j++) {
 
-                tokenizer.reset(accessor.getBuffer().array(), accessor.getTupleStartOffset(i)
-                        + accessor.getFieldSlotsLength() + accessor.getFieldStartOffset(i, tokenFields[j]), accessor
-                        .getFieldLength(i, tokenFields[j]));
+                tokenizer.reset(
+                        accessor.getBuffer().array(),
+                        accessor.getTupleStartOffset(i) + accessor.getFieldSlotsLength()
+                                + accessor.getFieldStartOffset(i, tokenFields[j]),
+                        accessor.getFieldLength(i, tokenFields[j]));
 
                 while (tokenizer.hasNext()) {
                     tokenizer.next();
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/SimpleConjunctiveSearcher.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/SimpleConjunctiveSearcher.java
index 6792b0f..6ad5a9c 100644
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/SimpleConjunctiveSearcher.java
+++ b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/SimpleConjunctiveSearcher.java
@@ -21,7 +21,7 @@
 import java.util.ArrayList;
 import java.util.List;
 
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -54,7 +54,7 @@
     private final IBinaryComparator[] valueCmps;
 
     private final BTree btree;
-    private final IHyracksContext ctx;
+    private final IHyracksStageletContext ctx;
     private final ArrayTupleBuilder resultTupleBuilder;
     private final FrameTupleAppender resultTupleAppender;
     private final FrameTupleAccessor resultFrameAccessor;
@@ -73,7 +73,7 @@
 
     private final IBinaryTokenizer queryTokenizer;
 
-    public SimpleConjunctiveSearcher(IHyracksContext ctx, BTree btree, RecordDescriptor btreeRecDesc,
+    public SimpleConjunctiveSearcher(IHyracksStageletContext ctx, BTree btree, RecordDescriptor btreeRecDesc,
             IBinaryTokenizer queryTokenizer, int numKeyFields, int numValueFields) {
         this.ctx = ctx;
         this.btree = btree;
@@ -84,10 +84,10 @@
         leafFrame = btree.getLeafFrameFactory().getFrame();
         interiorFrame = btree.getInteriorFrameFactory().getFrame();
         btreeCursor = new RangeSearchCursor(leafFrame);
-        resultTupleAppender = new FrameTupleAppender(ctx);
+        resultTupleAppender = new FrameTupleAppender(ctx.getFrameSize());
         resultTupleBuilder = new ArrayTupleBuilder(numValueFields);
-        newResultBuffers.add(ctx.getResourceManager().allocateFrame());
-        prevResultBuffers.add(ctx.getResourceManager().allocateFrame());
+        newResultBuffers.add(ctx.allocateFrame());
+        prevResultBuffers.add(ctx.allocateFrame());
 
         MultiComparator btreeCmp = btree.getMultiComparator();
 
@@ -113,7 +113,7 @@
 
         }
         RecordDescriptor valueRecDesc = new RecordDescriptor(valueSerde);
-        resultFrameAccessor = new FrameTupleAccessor(ctx, valueRecDesc);
+        resultFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(), valueRecDesc);
     }
 
     public void search(ITupleReference queryTuple, int queryFieldIndex) throws Exception {
@@ -124,8 +124,8 @@
 
         ArrayTupleBuilder queryTokenBuilder = new ArrayTupleBuilder(queryTokenRecDesc.getFields().length);
         DataOutput queryTokenDos = queryTokenBuilder.getDataOutput();
-        FrameTupleAppender queryTokenAppender = new FrameTupleAppender(ctx);
-        ByteBuffer queryTokenFrame = ctx.getResourceManager().allocateFrame();
+        FrameTupleAppender queryTokenAppender = new FrameTupleAppender(ctx.getFrameSize());
+        ByteBuffer queryTokenFrame = ctx.allocateFrame();
         queryTokenAppender.reset(queryTokenFrame, true);
 
         queryTokenizer.reset(queryTuple.getFieldData(queryFieldIndex), queryTuple.getFieldStart(queryFieldIndex),
@@ -146,7 +146,7 @@
                     queryTokenBuilder.getSize());
         }
 
-        FrameTupleAccessor queryTokenAccessor = new FrameTupleAccessor(ctx, queryTokenRecDesc);
+        FrameTupleAccessor queryTokenAccessor = new FrameTupleAccessor(ctx.getFrameSize(), queryTokenRecDesc);
         queryTokenAccessor.reset(queryTokenFrame);
         int numQueryTokens = queryTokenAccessor.getTupleCount();
 
@@ -204,7 +204,7 @@
                 resultTupleBuilder.getSize())) {
             newBufIdx++;
             if (newBufIdx >= newResultBuffers.size()) {
-                newResultBuffers.add(ctx.getResourceManager().allocateFrame());
+                newResultBuffers.add(ctx.allocateFrame());
             }
             newCurrentBuffer = newResultBuffers.get(newBufIdx);
             resultTupleAppender.reset(newCurrentBuffer, true);
@@ -245,12 +245,11 @@
             int cmp = 0;
             for (int i = 0; i < valueCmps.length; i++) {
                 int tupleFidx = numKeyFields + i;
-                cmp = valueCmps[i].compare(tuple.getFieldData(tupleFidx), tuple.getFieldStart(tupleFidx), tuple
-                        .getFieldLength(tupleFidx), resultFrameAccessor.getBuffer().array(), resultFrameAccessor
-                        .getTupleStartOffset(resultTidx)
-                        + resultFrameAccessor.getFieldSlotsLength()
-                        + resultFrameAccessor.getFieldStartOffset(resultTidx, i), resultFrameAccessor.getFieldLength(
-                        resultTidx, i));
+                cmp = valueCmps[i].compare(tuple.getFieldData(tupleFidx), tuple.getFieldStart(tupleFidx),
+                        tuple.getFieldLength(tupleFidx), resultFrameAccessor.getBuffer().array(),
+                        resultFrameAccessor.getTupleStartOffset(resultTidx) + resultFrameAccessor.getFieldSlotsLength()
+                                + resultFrameAccessor.getFieldStartOffset(resultTidx, i),
+                        resultFrameAccessor.getFieldLength(resultTidx, i));
                 if (cmp != 0)
                     break;
             }
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/IStorageManagerInterface.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/IStorageManagerInterface.java
index ad6f743..f64af2e 100644
--- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/IStorageManagerInterface.java
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/IStorageManagerInterface.java
@@ -16,11 +16,12 @@
 
 import java.io.Serializable;
 
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 
 public interface IStorageManagerInterface extends Serializable {
-    public IBufferCache getBufferCache();
+    public IBufferCache getBufferCache(IHyracksStageletContext ctx);
 
-    public IFileMapProvider getFileMapProvider();
+    public IFileMapProvider getFileMapProvider(IHyracksStageletContext ctx);
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java
index 7578f39..c2693ab 100644
--- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java
@@ -14,9 +14,6 @@
  */
 package edu.uci.ics.hyracks.storage.common.buffercache;
 
-import java.io.File;
-import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.Map;
@@ -30,7 +27,10 @@
 import java.util.logging.Logger;
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.common.file.FileHandle;
+import edu.uci.ics.hyracks.api.io.FileHandle;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapManager;
 
 public class BufferCache implements IBufferCacheInternal {
@@ -39,6 +39,7 @@
 
     private static final int MAX_VICTIMIZATION_TRY_COUNT = 3;
 
+    private final IIOManager ioManager;
     private final int pageSize;
     private final int numPages;
     private final CachedPage[] cachedPages;
@@ -46,12 +47,13 @@
     private final IPageReplacementStrategy pageReplacementStrategy;
     private final IFileMapManager fileMapManager;
     private final CleanerThread cleanerThread;
-    private final Map<Integer, FileHandle> fileInfoMap;
+    private final Map<Integer, BufferedFileHandle> fileInfoMap;
 
     private boolean closed;
 
-    public BufferCache(ICacheMemoryAllocator allocator, IPageReplacementStrategy pageReplacementStrategy,
-            IFileMapManager fileMapManager, int pageSize, int numPages) {
+    public BufferCache(IIOManager ioManager, ICacheMemoryAllocator allocator,
+            IPageReplacementStrategy pageReplacementStrategy, IFileMapManager fileMapManager, int pageSize, int numPages) {
+        this.ioManager = ioManager;
         this.pageSize = pageSize;
         this.numPages = numPages;
         pageReplacementStrategy.setBufferCache(this);
@@ -66,7 +68,7 @@
         }
         this.pageReplacementStrategy = pageReplacementStrategy;
         this.fileMapManager = fileMapManager;
-        fileInfoMap = new HashMap<Integer, FileHandle>();
+        fileInfoMap = new HashMap<Integer, BufferedFileHandle>();
         cleanerThread = new CleanerThread();
         cleanerThread.start();
         closed = false;
@@ -278,18 +280,15 @@
     }
 
     private void read(CachedPage cPage) throws HyracksDataException {
-        FileHandle fInfo = getFileInfo(cPage);
-        try {
-            cPage.buffer.clear();
-            fInfo.getFileChannel().read(cPage.buffer, (long) FileHandle.getPageId(cPage.dpid) * pageSize);
-        } catch (IOException e) {
-            throw new HyracksDataException(e);
-        }
+        BufferedFileHandle fInfo = getFileInfo(cPage);
+        cPage.buffer.clear();
+        ioManager.syncRead(fInfo.getFileHandle(), (long) BufferedFileHandle.getPageId(cPage.dpid) * pageSize,
+                cPage.buffer);
     }
 
-    private FileHandle getFileInfo(CachedPage cPage) throws HyracksDataException {
+    private BufferedFileHandle getFileInfo(CachedPage cPage) throws HyracksDataException {
         synchronized (fileInfoMap) {
-            FileHandle fInfo = fileInfoMap.get(FileHandle.getFileId(cPage.dpid));
+            BufferedFileHandle fInfo = fileInfoMap.get(BufferedFileHandle.getFileId(cPage.dpid));
             if (fInfo == null) {
                 throw new HyracksDataException("No such file mapped");
             }
@@ -298,14 +297,11 @@
     }
 
     private void write(CachedPage cPage) throws HyracksDataException {
-        FileHandle fInfo = getFileInfo(cPage);
-        try {
-            cPage.buffer.position(0);
-            cPage.buffer.limit(pageSize);
-            fInfo.getFileChannel().write(cPage.buffer, (long) FileHandle.getPageId(cPage.dpid) * pageSize);
-        } catch (IOException e) {
-            throw new HyracksDataException(e);
-        }
+        BufferedFileHandle fInfo = getFileInfo(cPage);
+        cPage.buffer.position(0);
+        cPage.buffer.limit(pageSize);
+        ioManager.syncWrite(fInfo.getFileHandle(), (long) BufferedFileHandle.getPageId(cPage.dpid) * pageSize,
+                cPage.buffer);
     }
 
     @Override
@@ -492,12 +488,12 @@
     }
 
     @Override
-    public void createFile(String fileName) throws HyracksDataException {
+    public void createFile(FileReference fileRef) throws HyracksDataException {
         if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Creating file: " + fileName + " in cache: " + this);
+            LOGGER.info("Creating file: " + fileRef + " in cache: " + this);
         }
         synchronized (fileInfoMap) {
-            fileMapManager.registerFile(fileName);
+            fileMapManager.registerFile(fileRef);
         }
     }
 
@@ -507,20 +503,13 @@
             LOGGER.info("Opening file: " + fileId + " in cache: " + this);
         }
         synchronized (fileInfoMap) {
-            FileHandle fInfo;
+            BufferedFileHandle fInfo;
             fInfo = fileInfoMap.get(fileId);
             if (fInfo == null) {
-                String fileName = fileMapManager.lookupFileName(fileId);
-                try {
-                    File f = new File(fileName);
-                    if (!f.exists()) {
-                        File dir = new File(f.getParent());
-                        dir.mkdirs();
-                    }
-                    fInfo = new FileHandle(fileId, new RandomAccessFile(f, "rw"));
-                } catch (IOException e) {
-                    throw new HyracksDataException(e);
-                }
+                FileReference fileRef = fileMapManager.lookupFileName(fileId);
+                FileHandle fh = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_WRITE,
+                        IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+                fInfo = new BufferedFileHandle(fileId, fh);
                 fileInfoMap.put(fileId, fInfo);
             }
             fInfo.incReferenceCount();
@@ -560,7 +549,7 @@
     }
 
     private boolean invalidateIfFileIdMatch(int fileId, CachedPage cPage) throws HyracksDataException {
-        if (FileHandle.getFileId(cPage.dpid) == fileId) {
+        if (BufferedFileHandle.getFileId(cPage.dpid) == fileId) {
             if (cPage.dirty.get()) {
                 write(cPage);
                 cPage.dirty.set(false);
@@ -581,14 +570,14 @@
             LOGGER.info("Closing file: " + fileId + " in cache: " + this);
         }
         synchronized (fileInfoMap) {
-            FileHandle fInfo = fileInfoMap.get(fileId);
+            BufferedFileHandle fInfo = fileInfoMap.get(fileId);
             if (fInfo == null) {
                 throw new HyracksDataException("Closing unopened file");
             }
             if (fInfo.decReferenceCount() <= 0) {
                 sweepAndFlush(fileId);
                 fileInfoMap.remove(fileId);
-                fInfo.close();
+                ioManager.close(fInfo.getFileHandle());
             }
         }
     }
@@ -599,7 +588,7 @@
             LOGGER.info("Deleting file: " + fileId + " in cache: " + this);
         }
         synchronized (fileInfoMap) {
-            FileHandle fInfo = fileInfoMap.get(fileId);
+            BufferedFileHandle fInfo = fileInfoMap.get(fileId);
             if (fInfo != null) {
                 throw new HyracksDataException("Deleting open file");
             }
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/IBufferCache.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/IBufferCache.java
index 2e7181e..b21c054 100644
--- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/IBufferCache.java
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/IBufferCache.java
@@ -15,9 +15,10 @@
 package edu.uci.ics.hyracks.storage.common.buffercache;
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
 
 public interface IBufferCache {
-    public void createFile(String fileName) throws HyracksDataException;
+    public void createFile(FileReference fileRef) throws HyracksDataException;
 
     public void openFile(int fileId) throws HyracksDataException;
 
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/FileHandle.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/BufferedFileHandle.java
similarity index 67%
rename from hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/FileHandle.java
rename to hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/BufferedFileHandle.java
index ddbedb4..1db0bf7 100644
--- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/FileHandle.java
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/BufferedFileHandle.java
@@ -14,45 +14,27 @@
  */
 package edu.uci.ics.hyracks.storage.common.file;
 
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.nio.channels.FileChannel;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileHandle;
 
-public class FileHandle {
+public class BufferedFileHandle {
     private final int fileId;
-    private final RandomAccessFile file;
+    private FileHandle handle;
     private final AtomicInteger refCount;
-    private FileChannel channel;
 
-    public FileHandle(int fileId, RandomAccessFile file) {
+    public BufferedFileHandle(int fileId, FileHandle handle) {
         this.fileId = fileId;
-        this.file = file;
+        this.handle = handle;
         refCount = new AtomicInteger();
-        channel = file.getChannel();
     }
 
     public int getFileId() {
         return fileId;
     }
 
-    public RandomAccessFile getFile() {
-        return file;
-    }
-
-    public FileChannel getFileChannel() {
-        return channel;
-    }
-
-    public void close() throws HyracksDataException {
-        try {
-            channel.close();
-            file.close();
-        } catch (IOException e) {
-            throw new HyracksDataException(e);
-        }
+    public FileHandle getFileHandle() {
+        return handle;
     }
 
     public int incReferenceCount() {
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/IFileMapManager.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/IFileMapManager.java
index d37e576..fa1ec3a 100644
--- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/IFileMapManager.java
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/IFileMapManager.java
@@ -15,6 +15,7 @@
 package edu.uci.ics.hyracks.storage.common.file;
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
 
 /**
  * Maintains the mapping between file names and file ids.
@@ -25,12 +26,12 @@
     /**
      * Register a new file name.
      * 
-     * @param fileName
-     *            - the name of the file to register
+     * @param fileRef
+     *            - file reference to register
      * @throws HyracksDataException
      *             - if a mapping for the file already exists.
      */
-    public void registerFile(String fileName) throws HyracksDataException;
+    public void registerFile(FileReference fileRef) throws HyracksDataException;
 
     /**
      * Unregister a file mapping
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/IFileMapProvider.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/IFileMapProvider.java
index 38d9f8c..5219675 100644
--- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/IFileMapProvider.java
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/IFileMapProvider.java
@@ -17,6 +17,7 @@
 import java.io.Serializable;
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
 
 public interface IFileMapProvider extends Serializable {
     /**
@@ -30,30 +31,30 @@
     /**
      * Indicates if a given file name is mapped.
      * 
-     * @param fileName
+     * @param fileRef
      * @return <code>true</code> if the given file name is mapped, <code>false</code> otherwise.
      */
-    public boolean isMapped(String fileName);
+    public boolean isMapped(FileReference fileRef);
 
     /**
      * Lookup the file id for a file name
      * 
-     * @param fileName
+     * @param fileRef
      *            - The file name whose id should be looked up.
      * @return The file id
      * @throws HyracksDataException
      *             - If the file name is not currently mapped in this manager.
      */
-    public int lookupFileId(String fileName) throws HyracksDataException;
+    public int lookupFileId(FileReference fileRef) throws HyracksDataException;
 
     /**
      * Lookup the file name for a file id
      * 
      * @param fileId
      *            - The file id whose name should be looked up.
-     * @return The file name
+     * @return The file reference
      * @throws HyracksDataException
      *             - If the file id is not mapped currently in this manager.
      */
-    public String lookupFileName(int fileId) throws HyracksDataException;
+    public FileReference lookupFileName(int fileId) throws HyracksDataException;
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/smi/TransientFileMapManager.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/smi/TransientFileMapManager.java
new file mode 100644
index 0000000..e4ddc08
--- /dev/null
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/smi/TransientFileMapManager.java
@@ -0,0 +1,57 @@
+package edu.uci.ics.hyracks.storage.common.smi;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapManager;
+
+public class TransientFileMapManager implements IFileMapManager {
+    private static final long serialVersionUID = 1L;
+
+    private Map<Integer, FileReference> id2nameMap = new HashMap<Integer, FileReference>();
+    private Map<FileReference, Integer> name2IdMap = new HashMap<FileReference, Integer>();
+    private int idCounter = 0;
+
+    @Override
+    public FileReference lookupFileName(int fileId) throws HyracksDataException {
+        FileReference fRef = id2nameMap.get(fileId);
+        if (fRef == null) {
+            throw new HyracksDataException("No mapping found for id: " + fileId);
+        }
+        return fRef;
+    }
+
+    @Override
+    public int lookupFileId(FileReference fileRef) throws HyracksDataException {
+        Integer fileId = name2IdMap.get(fileRef);
+        if (fileId == null) {
+            throw new HyracksDataException("No mapping found for name: " + fileRef);
+        }
+        return fileId;
+    }
+
+    @Override
+    public boolean isMapped(FileReference fileRef) {
+        return name2IdMap.containsKey(fileRef);
+    }
+
+    @Override
+    public boolean isMapped(int fileId) {
+        return id2nameMap.containsKey(fileId);
+    }
+
+    @Override
+    public void unregisterFile(int fileId) throws HyracksDataException {
+        FileReference fileRef = id2nameMap.remove(fileId);
+        name2IdMap.remove(fileRef);
+    }
+
+    @Override
+    public void registerFile(FileReference fileRef) throws HyracksDataException {
+        Integer fileId = idCounter++;
+        id2nameMap.put(fileId, fileRef);
+        name2IdMap.put(fileRef, fileId);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-test-support/.classpath b/hyracks/hyracks-test-support/.classpath
new file mode 100644
index 0000000..1f3c1ff
--- /dev/null
+++ b/hyracks/hyracks-test-support/.classpath
@@ -0,0 +1,7 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<classpath>
+	<classpathentry kind="src" output="target/classes" path="src/main/java"/>
+	<classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/JavaSE-1.6"/>
+	<classpathentry kind="con" path="org.maven.ide.eclipse.MAVEN2_CLASSPATH_CONTAINER"/>
+	<classpathentry kind="output" path="target/classes"/>
+</classpath>
diff --git a/hyracks/hyracks-test-support/.project b/hyracks/hyracks-test-support/.project
new file mode 100644
index 0000000..b0c27d2
--- /dev/null
+++ b/hyracks/hyracks-test-support/.project
@@ -0,0 +1,23 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<projectDescription>
+	<name>hyracks-test-support</name>
+	<comment></comment>
+	<projects>
+	</projects>
+	<buildSpec>
+		<buildCommand>
+			<name>org.eclipse.jdt.core.javabuilder</name>
+			<arguments>
+			</arguments>
+		</buildCommand>
+		<buildCommand>
+			<name>org.maven.ide.eclipse.maven2Builder</name>
+			<arguments>
+			</arguments>
+		</buildCommand>
+	</buildSpec>
+	<natures>
+		<nature>org.eclipse.jdt.core.javanature</nature>
+		<nature>org.maven.ide.eclipse.maven2Nature</nature>
+	</natures>
+</projectDescription>
diff --git a/hyracks/hyracks-test-support/.settings/org.eclipse.jdt.core.prefs b/hyracks/hyracks-test-support/.settings/org.eclipse.jdt.core.prefs
new file mode 100644
index 0000000..18b0312
--- /dev/null
+++ b/hyracks/hyracks-test-support/.settings/org.eclipse.jdt.core.prefs
@@ -0,0 +1,6 @@
+#Wed Jan 05 15:09:31 PST 2011
+eclipse.preferences.version=1
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.6
+org.eclipse.jdt.core.compiler.compliance=1.6
+org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning
+org.eclipse.jdt.core.compiler.source=1.6
diff --git a/hyracks/hyracks-test-support/.settings/org.maven.ide.eclipse.prefs b/hyracks/hyracks-test-support/.settings/org.maven.ide.eclipse.prefs
new file mode 100644
index 0000000..f462ae6
--- /dev/null
+++ b/hyracks/hyracks-test-support/.settings/org.maven.ide.eclipse.prefs
@@ -0,0 +1,9 @@
+#Wed Jan 05 15:09:30 PST 2011
+activeProfiles=
+eclipse.preferences.version=1
+fullBuildGoals=process-test-resources
+includeModules=false
+resolveWorkspaceProjects=true
+resourceFilterGoals=process-resources resources\:testResources
+skipCompilerPlugin=true
+version=1
diff --git a/hyracks/hyracks-test-support/pom.xml b/hyracks/hyracks-test-support/pom.xml
new file mode 100644
index 0000000..14c2275
--- /dev/null
+++ b/hyracks/hyracks-test-support/pom.xml
@@ -0,0 +1,47 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <groupId>edu.uci.ics.hyracks</groupId>
+  <artifactId>hyracks-test-support</artifactId>
+  <version>0.1.4-SNAPSHOT</version>
+
+  <parent>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks</artifactId>
+    <version>0.1.4-SNAPSHOT</version>
+  </parent>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>2.0.2</version>
+        <configuration>
+          <source>1.6</source>
+          <target>1.6</target>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+  <dependencies>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-control-nc</artifactId>
+  		<version>0.1.4-SNAPSHOT</version>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-storage-common</artifactId>
+  		<version>0.1.4-SNAPSHOT</version>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-storage-am-btree</artifactId>
+  		<version>0.1.4-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>
+  </dependencies>
+</project>
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeRegistryProvider.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestBTreeRegistryProvider.java
similarity index 60%
rename from hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeRegistryProvider.java
rename to hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestBTreeRegistryProvider.java
index ef4e85c..3ab0427 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeRegistryProvider.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestBTreeRegistryProvider.java
@@ -12,20 +12,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package edu.uci.ics.hyracks.test.support;
 
-package edu.uci.ics.hyracks.storage.am.btree.dataflow;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeRegistry;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.IBTreeRegistryProvider;
 
-public class BTreeRegistryProvider implements IBTreeRegistryProvider {
-
+public class TestBTreeRegistryProvider implements IBTreeRegistryProvider {
     private static final long serialVersionUID = 1L;
 
-    private static BTreeRegistry btreeRegistry = null;
-
     @Override
-    public synchronized BTreeRegistry getBTreeRegistry() {
-        if (btreeRegistry == null) {
-            btreeRegistry = new BTreeRegistry();
-        }
-        return btreeRegistry;
+    public BTreeRegistry getBTreeRegistry(IHyracksStageletContext ctx) {
+        return TestStorageManagerComponentHolder.getBTreeRegistry(ctx);
     }
-}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java
new file mode 100644
index 0000000..921cb55
--- /dev/null
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java
@@ -0,0 +1,94 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.test.support;
+
+import java.nio.ByteBuffer;
+import java.util.UUID;
+
+import edu.uci.ics.hyracks.api.application.INCApplicationContext;
+import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
+import edu.uci.ics.hyracks.api.resources.IDeallocatable;
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
+import edu.uci.ics.hyracks.control.nc.io.ManagedWorkspaceFileFactory;
+import edu.uci.ics.hyracks.control.nc.job.profiling.CounterContext;
+
+public class TestJobletContext implements IHyracksJobletContext {
+    private final INCApplicationContext appContext;
+    private UUID jobId;
+    private int attempt;
+    private ManagedWorkspaceFileFactory fileFactory;
+
+    public TestJobletContext(INCApplicationContext appContext, UUID jobId, int attempt) throws HyracksException {
+        this.appContext = appContext;
+        this.jobId = jobId;
+        this.attempt = attempt;
+        fileFactory = new ManagedWorkspaceFileFactory(this, (IOManager) getIOManager());
+    }
+
+    @Override
+    public ByteBuffer allocateFrame() {
+        return appContext.getRootContext().allocateFrame();
+    }
+
+    @Override
+    public int getFrameSize() {
+        return appContext.getRootContext().getFrameSize();
+    }
+
+    @Override
+    public IIOManager getIOManager() {
+        return appContext.getRootContext().getIOManager();
+    }
+
+    @Override
+    public FileReference createWorkspaceFile(String prefix) throws HyracksDataException {
+        return fileFactory.createWorkspaceFile(prefix);
+    }
+
+    @Override
+    public ICounterContext getCounterContext() {
+        return new CounterContext(jobId + "." + attempt);
+    }
+
+    @Override
+    public void registerDeallocatable(final IDeallocatable deallocatable) {
+        Runtime.getRuntime().addShutdownHook(new Thread() {
+            @Override
+            public void run() {
+                deallocatable.deallocate();
+            }
+        });
+    }
+
+    @Override
+    public INCApplicationContext getApplicationContext() {
+        return appContext;
+    }
+
+    @Override
+    public UUID getJobId() {
+        return jobId;
+    }
+
+    @Override
+    public int getAttempt() {
+        return attempt;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java
new file mode 100644
index 0000000..0bdf919
--- /dev/null
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.test.support;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.application.INCApplicationContext;
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+
+public class TestNCApplicationContext implements INCApplicationContext {
+    private IHyracksRootContext rootCtx;
+    private Serializable distributedState;
+    private Object appObject;
+
+    public TestNCApplicationContext(IHyracksRootContext rootCtx) {
+        this.rootCtx = rootCtx;
+    }
+
+    @Override
+    public ClassLoader getClassLoader() {
+        return getClass().getClassLoader();
+    }
+
+    @Override
+    public Serializable getDestributedState() {
+        return distributedState;
+    }
+
+    @Override
+    public IHyracksRootContext getRootContext() {
+        return rootCtx;
+    }
+
+    @Override
+    public void setDistributedState(Serializable state) {
+        distributedState = state;
+    }
+
+    @Override
+    public void setApplicationObject(Object object) {
+        this.appObject = object;
+    }
+
+    @Override
+    public Object getApplicationObject() {
+        return appObject;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestRootContext.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestRootContext.java
new file mode 100644
index 0000000..93409ef
--- /dev/null
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestRootContext.java
@@ -0,0 +1,54 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.test.support;
+
+import java.io.File;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Executors;
+
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.io.IODeviceHandle;
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
+
+public class TestRootContext implements IHyracksRootContext {
+    private int frameSize;
+    private IOManager ioManager;
+
+    public TestRootContext(int frameSize) throws HyracksException {
+        this.frameSize = frameSize;
+        List<IODeviceHandle> devices = new ArrayList<IODeviceHandle>();
+        devices.add(new IODeviceHandle(new File(System.getProperty("java.io.tmpdir")), "."));
+        ioManager = new IOManager(devices, Executors.newCachedThreadPool());
+    }
+
+    @Override
+    public ByteBuffer allocateFrame() {
+        return ByteBuffer.allocate(frameSize);
+    }
+
+    @Override
+    public int getFrameSize() {
+        return frameSize;
+    }
+
+    @Override
+    public IIOManager getIOManager() {
+        return ioManager;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStageletContext.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStageletContext.java
new file mode 100644
index 0000000..d28911b
--- /dev/null
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStageletContext.java
@@ -0,0 +1,87 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.test.support;
+
+import java.nio.ByteBuffer;
+import java.util.UUID;
+
+import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
+import edu.uci.ics.hyracks.api.resources.IDeallocatable;
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
+import edu.uci.ics.hyracks.control.nc.io.ManagedWorkspaceFileFactory;
+import edu.uci.ics.hyracks.control.nc.job.profiling.CounterContext;
+
+public class TestStageletContext implements IHyracksStageletContext {
+    private final IHyracksJobletContext jobletContext;
+    private UUID stageId;
+    private ManagedWorkspaceFileFactory fileFactory;
+
+    public TestStageletContext(IHyracksJobletContext jobletContext, UUID stageId) throws HyracksException {
+        this.jobletContext = jobletContext;
+        this.stageId = stageId;
+        fileFactory = new ManagedWorkspaceFileFactory(this, (IOManager) getIOManager());
+    }
+
+    @Override
+    public ByteBuffer allocateFrame() {
+        return jobletContext.allocateFrame();
+    }
+
+    @Override
+    public int getFrameSize() {
+        return jobletContext.getFrameSize();
+    }
+
+    @Override
+    public IIOManager getIOManager() {
+        return jobletContext.getIOManager();
+    }
+
+    @Override
+    public FileReference createWorkspaceFile(String prefix) throws HyracksDataException {
+        return fileFactory.createWorkspaceFile(prefix);
+    }
+
+    @Override
+    public IHyracksJobletContext getJobletContext() {
+        return jobletContext;
+    }
+
+    @Override
+    public UUID getStageId() {
+        return stageId;
+    }
+
+    @Override
+    public ICounterContext getCounterContext() {
+        return new CounterContext(jobletContext.getJobId() + "." + jobletContext.getAttempt() + "." + stageId);
+    }
+
+    @Override
+    public void registerDeallocatable(final IDeallocatable deallocatable) {
+        Runtime.getRuntime().addShutdownHook(new Thread() {
+            @Override
+            public void run() {
+                deallocatable.deallocate();
+            }
+        });
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java
new file mode 100644
index 0000000..5338d5e
--- /dev/null
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.test.support;
+
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeRegistry;
+import edu.uci.ics.hyracks.storage.common.buffercache.BufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.ClockPageReplacementStrategy;
+import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICacheMemoryAllocator;
+import edu.uci.ics.hyracks.storage.common.buffercache.IPageReplacementStrategy;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapManager;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.storage.common.smi.TransientFileMapManager;
+
+public class TestStorageManagerComponentHolder {
+    private static IBufferCache bufferCache;
+    private static IFileMapProvider fileMapProvider;
+    private static BTreeRegistry btreeRegistry;
+
+    private static int pageSize;
+    private static int numPages;
+
+    public static void init(int pageSize, int numPages) {
+        TestStorageManagerComponentHolder.pageSize = pageSize;
+        TestStorageManagerComponentHolder.numPages = numPages;
+        bufferCache = null;
+        fileMapProvider = null;
+        btreeRegistry = null;
+    }
+
+    public synchronized static IBufferCache getBufferCache(IHyracksStageletContext ctx) {
+        if (bufferCache == null) {
+            ICacheMemoryAllocator allocator = new HeapBufferAllocator();
+            IPageReplacementStrategy prs = new ClockPageReplacementStrategy();
+            IFileMapProvider fileMapProvider = getFileMapProvider(ctx);
+            bufferCache = new BufferCache(ctx.getIOManager(), allocator, prs, (IFileMapManager) fileMapProvider,
+                    pageSize, numPages);
+        }
+        return bufferCache;
+    }
+
+    public synchronized static IFileMapProvider getFileMapProvider(IHyracksStageletContext ctx) {
+        if (fileMapProvider == null) {
+            fileMapProvider = new TransientFileMapManager();
+        }
+        return fileMapProvider;
+    }
+
+    public synchronized static BTreeRegistry getBTreeRegistry(IHyracksStageletContext ctx) {
+        if (btreeRegistry == null) {
+            btreeRegistry = new BTreeRegistry();
+        }
+        return btreeRegistry;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/SimpleStorageManager.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerInterface.java
similarity index 65%
rename from hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/SimpleStorageManager.java
rename to hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerInterface.java
index 8348bb9..e923cdd 100644
--- a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/SimpleStorageManager.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerInterface.java
@@ -12,27 +12,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package edu.uci.ics.hyracks.test.support;
 
-package edu.uci.ics.hyracks.examples.btree.helper;
-
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 
-public class SimpleStorageManager implements IStorageManagerInterface {
-
+public class TestStorageManagerInterface implements IStorageManagerInterface {
     private static final long serialVersionUID = 1L;
 
-    public static SimpleStorageManager INSTANCE = new SimpleStorageManager();
-
     @Override
-    public IBufferCache getBufferCache() {
-        return RuntimeContext.getInstance().getBufferCache();
+    public IBufferCache getBufferCache(IHyracksStageletContext ctx) {
+        return TestStorageManagerComponentHolder.getBufferCache(ctx);
     }
 
     @Override
-    public IFileMapProvider getFileMapProvider() {
-        return RuntimeContext.getInstance().getFileMapManager();
+    public IFileMapProvider getFileMapProvider(IHyracksStageletContext ctx) {
+        return TestStorageManagerComponentHolder.getFileMapProvider(ctx);
     }
-
-}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestUtils.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestUtils.java
new file mode 100644
index 0000000..2edeaff
--- /dev/null
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestUtils.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.test.support;
+
+import java.util.UUID;
+
+import edu.uci.ics.hyracks.api.application.INCApplicationContext;
+import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+
+public class TestUtils {
+    public static IHyracksStageletContext create(int frameSize) {
+        try {
+            IHyracksRootContext rootCtx = new TestRootContext(frameSize);
+            INCApplicationContext appCtx = new TestNCApplicationContext(rootCtx);
+            IHyracksJobletContext jobletCtx = new TestJobletContext(appCtx, UUID.randomUUID(), 0);
+            IHyracksStageletContext stageletCtx = new TestStageletContext(jobletCtx, UUID.randomUUID());
+            return stageletCtx;
+        } catch (HyracksException e) {
+            throw new RuntimeException(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-tests/.project b/hyracks/hyracks-tests/.project
new file mode 100644
index 0000000..198463d
--- /dev/null
+++ b/hyracks/hyracks-tests/.project
@@ -0,0 +1,17 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<projectDescription>
+	<name>hyracks-tests</name>
+	<comment></comment>
+	<projects>
+	</projects>
+	<buildSpec>
+		<buildCommand>
+			<name>org.maven.ide.eclipse.maven2Builder</name>
+			<arguments>
+			</arguments>
+		</buildCommand>
+	</buildSpec>
+	<natures>
+		<nature>org.maven.ide.eclipse.maven2Nature</nature>
+	</natures>
+</projectDescription>
diff --git a/hyracks/hyracks-tests/.settings/org.maven.ide.eclipse.prefs b/hyracks/hyracks-tests/.settings/org.maven.ide.eclipse.prefs
new file mode 100644
index 0000000..99b89a6
--- /dev/null
+++ b/hyracks/hyracks-tests/.settings/org.maven.ide.eclipse.prefs
@@ -0,0 +1,9 @@
+#Thu Jan 06 11:27:16 PST 2011
+activeProfiles=
+eclipse.preferences.version=1
+fullBuildGoals=process-test-resources
+includeModules=false
+resolveWorkspaceProjects=true
+resourceFilterGoals=process-resources resources\:testResources
+skipCompilerPlugin=true
+version=1
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/.classpath b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/.classpath
new file mode 100644
index 0000000..e44aa2f
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/.classpath
@@ -0,0 +1,7 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<classpath>
+	<classpathentry kind="src" path="src/test/java"/>
+	<classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/JavaSE-1.6"/>
+	<classpathentry kind="con" path="org.maven.ide.eclipse.MAVEN2_CLASSPATH_CONTAINER"/>
+	<classpathentry kind="output" path="target/classes"/>
+</classpath>
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/.project b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/.project
new file mode 100644
index 0000000..bc6bc56
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/.project
@@ -0,0 +1,23 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<projectDescription>
+	<name>hyracks-storage-am-btree-test</name>
+	<comment></comment>
+	<projects>
+	</projects>
+	<buildSpec>
+		<buildCommand>
+			<name>org.eclipse.jdt.core.javabuilder</name>
+			<arguments>
+			</arguments>
+		</buildCommand>
+		<buildCommand>
+			<name>org.maven.ide.eclipse.maven2Builder</name>
+			<arguments>
+			</arguments>
+		</buildCommand>
+	</buildSpec>
+	<natures>
+		<nature>org.eclipse.jdt.core.javanature</nature>
+		<nature>org.maven.ide.eclipse.maven2Nature</nature>
+	</natures>
+</projectDescription>
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/.settings/org.eclipse.jdt.core.prefs b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/.settings/org.eclipse.jdt.core.prefs
new file mode 100644
index 0000000..3cd389e
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/.settings/org.eclipse.jdt.core.prefs
@@ -0,0 +1,6 @@
+#Thu Jan 06 11:27:16 PST 2011
+eclipse.preferences.version=1
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.6
+org.eclipse.jdt.core.compiler.compliance=1.6
+org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning
+org.eclipse.jdt.core.compiler.source=1.6
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/.settings/org.maven.ide.eclipse.prefs b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/.settings/org.maven.ide.eclipse.prefs
new file mode 100644
index 0000000..99b89a6
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/.settings/org.maven.ide.eclipse.prefs
@@ -0,0 +1,9 @@
+#Thu Jan 06 11:27:16 PST 2011
+activeProfiles=
+eclipse.preferences.version=1
+fullBuildGoals=process-test-resources
+includeModules=false
+resolveWorkspaceProjects=true
+resourceFilterGoals=process-resources resources\:testResources
+skipCompilerPlugin=true
+version=1
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
new file mode 100644
index 0000000..7e78c2a
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
@@ -0,0 +1,55 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <groupId>edu.uci.ics.hyracks</groupId>
+  <artifactId>hyracks-storage-am-btree-test</artifactId>
+  <version>0.1.4-SNAPSHOT</version>
+
+  <parent>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks-tests</artifactId>
+    <version>0.1.4-SNAPSHOT</version>
+  </parent>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>2.0.2</version>
+        <configuration>
+          <source>1.6</source>
+          <target>1.6</target>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+  <dependencies>
+  	<dependency>
+  		<groupId>junit</groupId>
+  		<artifactId>junit</artifactId>
+  		<version>4.8.1</version>
+  		<type>jar</type>
+  		<scope>test</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-control-nc</artifactId>
+  		<version>0.1.4-SNAPSHOT</version>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-storage-am-btree</artifactId>
+  		<version>0.1.4-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-test-support</artifactId>
+  		<version>0.1.4-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>test</scope>
+  	</dependency>
+  </dependencies>
+</project>
diff --git a/hyracks/hyracks-storage-am-btree/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeFieldPrefixNSMTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeFieldPrefixNSMTest.java
similarity index 85%
rename from hyracks/hyracks-storage-am-btree/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeFieldPrefixNSMTest.java
rename to hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeFieldPrefixNSMTest.java
index ff8160c..0798c22 100644
--- a/hyracks/hyracks-storage-am-btree/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeFieldPrefixNSMTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeFieldPrefixNSMTest.java
@@ -16,6 +16,7 @@
 package edu.uci.ics.hyracks.storage.am.btree;
 
 import java.io.DataOutput;
+import java.io.File;
 import java.nio.ByteBuffer;
 import java.util.Random;
 
@@ -23,14 +24,14 @@
 import org.junit.Test;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.control.nc.runtime.RootHyracksContext;
+import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
@@ -38,7 +39,6 @@
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import edu.uci.ics.hyracks.storage.am.btree.api.DummySMI;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeTupleWriter;
 import edu.uci.ics.hyracks.storage.am.btree.api.IPrefixSlotManager;
 import edu.uci.ics.hyracks.storage.am.btree.frames.FieldPrefixNSMLeafFrame;
@@ -49,8 +49,10 @@
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ICacheMemoryAllocator;
 import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
-import edu.uci.ics.hyracks.storage.common.file.FileHandle;
+import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+import edu.uci.ics.hyracks.test.support.TestUtils;
 
 public class BTreeFieldPrefixNSMTest {
 
@@ -80,20 +82,20 @@
         }
     }
 
-    private ITupleReference createTuple(int f0, int f1, int f2, boolean print) throws HyracksDataException {
+    private ITupleReference createTuple(IHyracksStageletContext ctx, int f0, int f1, int f2, boolean print)
+            throws HyracksDataException {
         if (print)
             System.out.println("CREATING: " + f0 + " " + f1 + " " + f2);
 
-        IHyracksContext ctx = new RootHyracksContext(HYRACKS_FRAME_SIZE);
-        ByteBuffer buf = ctx.getResourceManager().allocateFrame();
-        FrameTupleAppender appender = new FrameTupleAppender(ctx);
+        ByteBuffer buf = ctx.allocateFrame();
+        FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
         ArrayTupleBuilder tb = new ArrayTupleBuilder(3);
         DataOutput dos = tb.getDataOutput();
 
         ISerializerDeserializer[] recDescSers = { IntegerSerializerDeserializer.INSTANCE,
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
         RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
-        IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx, recDesc);
+        IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
         accessor.reset(buf);
         FrameTupleReference tuple = new FrameTupleReference();
 
@@ -115,13 +117,13 @@
 
     @Test
     public void test01() throws Exception {
-
-        DummySMI smi = new DummySMI(PAGE_SIZE, NUM_PAGES);
-        IBufferCache bufferCache = smi.getBufferCache();
-        IFileMapProvider fmp = smi.getFileMapProvider();
-        String fileName = tmpDir + "/" + "btreetest.bin";
-        bufferCache.createFile(fileName);
-        int fileId = fmp.lookupFileId(fileName);
+        IHyracksStageletContext stageletCtx = TestUtils.create(HYRACKS_FRAME_SIZE);
+        TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES);
+        IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(stageletCtx);
+        IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(stageletCtx);
+        FileReference file = new FileReference(new File(tmpDir + "/" + "btreetest.bin"));
+        bufferCache.createFile(file);
+        int fileId = fmp.lookupFileId(file);
         bufferCache.openFile(fileId);
 
         // declare fields
@@ -146,7 +148,7 @@
         Random rnd = new Random();
         rnd.setSeed(50);
 
-        ICachedPage page = bufferCache.pin(FileHandle.getDiskPageId(fileId, 0), false);
+        ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, 0), false);
         try {
 
             IPrefixSlotManager slotManager = new FieldPrefixSlotManager();
@@ -177,7 +179,7 @@
                 int b = rnd.nextInt() % smallMax;
                 int c = i;
 
-                ITupleReference tuple = createTuple(a, b, c, false);
+                ITupleReference tuple = createTuple(stageletCtx, a, b, c, false);
                 try {
                     frame.insert(tuple, cmp);
                 } catch (BTreeException e) {
@@ -212,7 +214,8 @@
                 if ((i + 1) % 100 == 0)
                     print("DELETING " + (i + 1) + " / " + numRecords + "\n");
 
-                ITupleReference tuple = createTuple(savedFields[i][0], savedFields[i][1], savedFields[i][2], false);
+                ITupleReference tuple = createTuple(stageletCtx, savedFields[i][0], savedFields[i][1],
+                        savedFields[i][2], false);
                 try {
                     frame.delete(tuple, cmp, true);
                 } catch (Exception e) {
@@ -240,4 +243,4 @@
         bufferCache.closeFile(fileId);
         bufferCache.close();
     }
-}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-btree/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java
similarity index 92%
rename from hyracks/hyracks-storage-am-btree/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java
rename to hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java
index 74dabe0..e725f7c 100644
--- a/hyracks/hyracks-storage-am-btree/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java
@@ -16,19 +16,20 @@
 package edu.uci.ics.hyracks.storage.am.btree;
 
 import java.io.DataOutput;
+import java.io.File;
 import java.nio.ByteBuffer;
 import java.util.Random;
 
 import org.junit.Test;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
-import edu.uci.ics.hyracks.control.nc.runtime.RootHyracksContext;
+import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
@@ -38,7 +39,6 @@
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import edu.uci.ics.hyracks.storage.am.btree.api.DummySMI;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeCursor;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
@@ -62,6 +62,8 @@
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ICacheMemoryAllocator;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+import edu.uci.ics.hyracks.test.support.TestUtils;
 
 @SuppressWarnings("unchecked")
 public class BTreeTest {
@@ -71,6 +73,7 @@
     private static final int HYRACKS_FRAME_SIZE = 128;
 
     private String tmpDir = System.getProperty("java.io.tmpdir");
+    IHyracksStageletContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
 
     // to help with the logger madness
     private void print(String str) {
@@ -102,12 +105,12 @@
 
         print("FIXED-LENGTH KEY TEST\n");
 
-        DummySMI smi = new DummySMI(PAGE_SIZE, NUM_PAGES);
-        IBufferCache bufferCache = smi.getBufferCache();
-        IFileMapProvider fmp = smi.getFileMapProvider();
-        String fileName = tmpDir + "/" + "btreetest.bin";
-        bufferCache.createFile(fileName);
-        int fileId = fmp.lookupFileId(fileName);
+        TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES);
+        IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+        IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+        FileReference file = new FileReference(new File(tmpDir + "/" + "btreetest.bin"));
+        bufferCache.createFile(file);
+        int fileId = fmp.lookupFileId(file);
         bufferCache.openFile(fileId);
 
         // declare fields
@@ -145,16 +148,15 @@
 
         print("INSERTING INTO TREE\n");
 
-        IHyracksContext ctx = new RootHyracksContext(HYRACKS_FRAME_SIZE);
-        ByteBuffer frame = ctx.getResourceManager().allocateFrame();
-        FrameTupleAppender appender = new FrameTupleAppender(ctx);
+        ByteBuffer frame = ctx.allocateFrame();
+        FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
         ArrayTupleBuilder tb = new ArrayTupleBuilder(cmp.getFieldCount());
         DataOutput dos = tb.getDataOutput();
 
         ISerializerDeserializer[] recDescSers = { IntegerSerializerDeserializer.INSTANCE,
                 IntegerSerializerDeserializer.INSTANCE };
         RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
-        IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx, recDesc);
+        IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
         accessor.reset(frame);
         FrameTupleReference tuple = new FrameTupleReference();
 
@@ -256,7 +258,7 @@
 
         ISerializerDeserializer[] keyDescSers = { IntegerSerializerDeserializer.INSTANCE };
         RecordDescriptor keyDesc = new RecordDescriptor(keyDescSers);
-        IFrameTupleAccessor keyAccessor = new FrameTupleAccessor(ctx, keyDesc);
+        IFrameTupleAccessor keyAccessor = new FrameTupleAccessor(ctx.getFrameSize(), keyDesc);
         keyAccessor.reset(frame);
 
         appender.reset(frame, true);
@@ -317,12 +319,12 @@
 
         print("COMPOSITE KEY TEST\n");
 
-        DummySMI smi = new DummySMI(PAGE_SIZE, NUM_PAGES);
-        IBufferCache bufferCache = smi.getBufferCache();
-        IFileMapProvider fmp = smi.getFileMapProvider();
-        String fileName = tmpDir + "/" + "btreetest.bin";
-        bufferCache.createFile(fileName);
-        int fileId = fmp.lookupFileId(fileName);
+        TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES);
+        IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+        IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+        FileReference file = new FileReference(new File(tmpDir + "/" + "btreetest.bin"));
+        bufferCache.createFile(file);
+        int fileId = fmp.lookupFileId(file);
         bufferCache.openFile(fileId);
 
         // declare fields
@@ -362,16 +364,15 @@
 
         print("INSERTING INTO TREE\n");
 
-        IHyracksContext ctx = new RootHyracksContext(HYRACKS_FRAME_SIZE);
-        ByteBuffer frame = ctx.getResourceManager().allocateFrame();
-        FrameTupleAppender appender = new FrameTupleAppender(ctx);
+        ByteBuffer frame = ctx.allocateFrame();
+        FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
         ArrayTupleBuilder tb = new ArrayTupleBuilder(cmp.getFieldCount());
         DataOutput dos = tb.getDataOutput();
 
         ISerializerDeserializer[] recDescSers = { IntegerSerializerDeserializer.INSTANCE,
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
         RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
-        IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx, recDesc);
+        IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
         accessor.reset(frame);
         FrameTupleReference tuple = new FrameTupleReference();
 
@@ -440,7 +441,7 @@
 
         ISerializerDeserializer[] keyDescSers = { IntegerSerializerDeserializer.INSTANCE };
         RecordDescriptor keyDesc = new RecordDescriptor(keyDescSers);
-        IFrameTupleAccessor keyAccessor = new FrameTupleAccessor(ctx, keyDesc);
+        IFrameTupleAccessor keyAccessor = new FrameTupleAccessor(ctx.getFrameSize(), keyDesc);
         keyAccessor.reset(frame);
 
         appender.reset(frame, true);
@@ -507,12 +508,12 @@
 
         print("VARIABLE-LENGTH KEY TEST\n");
 
-        DummySMI smi = new DummySMI(PAGE_SIZE, NUM_PAGES);
-        IBufferCache bufferCache = smi.getBufferCache();
-        IFileMapProvider fmp = smi.getFileMapProvider();
-        String fileName = tmpDir + "/" + "btreetest.bin";
-        bufferCache.createFile(fileName);
-        int fileId = fmp.lookupFileId(fileName);
+        TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES);
+        IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+        IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+        FileReference file = new FileReference(new File(tmpDir + "/" + "btreetest.bin"));
+        bufferCache.createFile(file);
+        int fileId = fmp.lookupFileId(file);
         bufferCache.openFile(fileId);
 
         // declare fields
@@ -546,16 +547,15 @@
         Random rnd = new Random();
         rnd.setSeed(50);
 
-        IHyracksContext ctx = new RootHyracksContext(HYRACKS_FRAME_SIZE);
-        ByteBuffer frame = ctx.getResourceManager().allocateFrame();
-        FrameTupleAppender appender = new FrameTupleAppender(ctx);
+        ByteBuffer frame = ctx.allocateFrame();
+        FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
         ArrayTupleBuilder tb = new ArrayTupleBuilder(cmp.getFieldCount());
         DataOutput dos = tb.getDataOutput();
 
         ISerializerDeserializer[] recDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
                 UTF8StringSerializerDeserializer.INSTANCE };
         RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
-        IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx, recDesc);
+        IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
         accessor.reset(frame);
         FrameTupleReference tuple = new FrameTupleReference();
 
@@ -624,7 +624,7 @@
 
         ISerializerDeserializer[] keyDescSers = { UTF8StringSerializerDeserializer.INSTANCE };
         RecordDescriptor keyDesc = new RecordDescriptor(keyDescSers);
-        IFrameTupleAccessor keyAccessor = new FrameTupleAccessor(ctx, keyDesc);
+        IFrameTupleAccessor keyAccessor = new FrameTupleAccessor(ctx.getFrameSize(), keyDesc);
         keyAccessor.reset(frame);
 
         appender.reset(frame, true);
@@ -686,12 +686,12 @@
 
         print("DELETION TEST\n");
 
-        DummySMI smi = new DummySMI(PAGE_SIZE, NUM_PAGES);
-        IBufferCache bufferCache = smi.getBufferCache();
-        IFileMapProvider fmp = smi.getFileMapProvider();
-        String fileName = tmpDir + "/" + "btreetest.bin";
-        bufferCache.createFile(fileName);
-        int fileId = fmp.lookupFileId(fileName);
+        TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES);
+        IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+        IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+        FileReference file = new FileReference(new File(tmpDir + "/" + "btreetest.bin"));
+        bufferCache.createFile(file);
+        int fileId = fmp.lookupFileId(file);
         bufferCache.openFile(fileId);
 
         // declare fields
@@ -725,16 +725,15 @@
         Random rnd = new Random();
         rnd.setSeed(50);
 
-        IHyracksContext ctx = new RootHyracksContext(HYRACKS_FRAME_SIZE);
-        ByteBuffer frame = ctx.getResourceManager().allocateFrame();
-        FrameTupleAppender appender = new FrameTupleAppender(ctx);
+        ByteBuffer frame = ctx.allocateFrame();
+        FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
         ArrayTupleBuilder tb = new ArrayTupleBuilder(cmp.getFieldCount());
         DataOutput dos = tb.getDataOutput();
 
         ISerializerDeserializer[] recDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
                 UTF8StringSerializerDeserializer.INSTANCE };
         RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
-        IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx, recDesc);
+        IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
         accessor.reset(frame);
         FrameTupleReference tuple = new FrameTupleReference();
 
@@ -852,12 +851,12 @@
 
         print("BULK LOAD TEST\n");
 
-        DummySMI smi = new DummySMI(PAGE_SIZE, NUM_PAGES);
-        IBufferCache bufferCache = smi.getBufferCache();
-        IFileMapProvider fmp = smi.getFileMapProvider();
-        String fileName = tmpDir + "/" + "btreetest.bin";
-        bufferCache.createFile(fileName);
-        int fileId = fmp.lookupFileId(fileName);
+        TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES);
+        IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+        IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+        FileReference file = new FileReference(new File(tmpDir + "/" + "btreetest.bin"));
+        bufferCache.createFile(file);
+        int fileId = fmp.lookupFileId(file);
         bufferCache.openFile(fileId);
 
         // declare fields
@@ -893,16 +892,15 @@
         Random rnd = new Random();
         rnd.setSeed(50);
 
-        IHyracksContext ctx = new RootHyracksContext(HYRACKS_FRAME_SIZE);
-        ByteBuffer frame = ctx.getResourceManager().allocateFrame();
-        FrameTupleAppender appender = new FrameTupleAppender(ctx);
+        ByteBuffer frame = ctx.allocateFrame();
+        FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
         ArrayTupleBuilder tb = new ArrayTupleBuilder(cmp.getFieldCount());
         DataOutput dos = tb.getDataOutput();
 
         ISerializerDeserializer[] recDescSers = { IntegerSerializerDeserializer.INSTANCE,
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
         RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
-        IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx, recDesc);
+        IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
         accessor.reset(frame);
         FrameTupleReference tuple = new FrameTupleReference();
 
@@ -948,7 +946,7 @@
 
         ISerializerDeserializer[] keyDescSers = { IntegerSerializerDeserializer.INSTANCE };
         RecordDescriptor keyDesc = new RecordDescriptor(keyDescSers);
-        IFrameTupleAccessor keyAccessor = new FrameTupleAccessor(ctx, keyDesc);
+        IFrameTupleAccessor keyAccessor = new FrameTupleAccessor(ctx.getFrameSize(), keyDesc);
         keyAccessor.reset(frame);
 
         appender.reset(frame, true);
@@ -1009,12 +1007,12 @@
 
         print("TIME-INTERVAL INTERSECTION DEMO\n");
 
-        DummySMI smi = new DummySMI(PAGE_SIZE, NUM_PAGES);
-        IBufferCache bufferCache = smi.getBufferCache();
-        IFileMapProvider fmp = smi.getFileMapProvider();
-        String fileName = tmpDir + "/" + "btreetest.bin";
-        bufferCache.createFile(fileName);
-        int fileId = fmp.lookupFileId(fileName);
+        TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES);
+        IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+        IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+        FileReference file = new FileReference(new File(tmpDir + "/" + "btreetest.bin"));
+        bufferCache.createFile(file);
+        int fileId = fmp.lookupFileId(file);
         bufferCache.openFile(fileId);
 
         // declare fields
@@ -1049,16 +1047,15 @@
         Random rnd = new Random();
         rnd.setSeed(50);
 
-        IHyracksContext ctx = new RootHyracksContext(HYRACKS_FRAME_SIZE);
-        ByteBuffer frame = ctx.getResourceManager().allocateFrame();
-        FrameTupleAppender appender = new FrameTupleAppender(ctx);
+        ByteBuffer frame = ctx.allocateFrame();
+        FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
         ArrayTupleBuilder tb = new ArrayTupleBuilder(cmp.getFieldCount());
         DataOutput dos = tb.getDataOutput();
 
         ISerializerDeserializer[] recDescSers = { IntegerSerializerDeserializer.INSTANCE,
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
         RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
-        IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx, recDesc);
+        IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
         accessor.reset(frame);
         FrameTupleReference tuple = new FrameTupleReference();
 
@@ -1166,7 +1163,7 @@
         ISerializerDeserializer[] keyDescSers = { IntegerSerializerDeserializer.INSTANCE,
                 IntegerSerializerDeserializer.INSTANCE };
         RecordDescriptor keyDesc = new RecordDescriptor(keyDescSers);
-        IFrameTupleAccessor keyAccessor = new FrameTupleAccessor(ctx, keyDesc);
+        IFrameTupleAccessor keyAccessor = new FrameTupleAccessor(ctx.getFrameSize(), keyDesc);
         keyAccessor.reset(frame);
 
         appender.reset(frame, true);
diff --git a/hyracks/hyracks-storage-am-btree/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java
similarity index 91%
rename from hyracks/hyracks-storage-am-btree/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java
rename to hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java
index 7ec564a..1651d56 100644
--- a/hyracks/hyracks-storage-am-btree/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java
@@ -19,6 +19,7 @@
 import java.io.DataInput;
 import java.io.DataInputStream;
 import java.io.DataOutput;
+import java.io.File;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -30,14 +31,14 @@
 import org.junit.Test;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.control.nc.runtime.RootHyracksContext;
+import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
@@ -45,7 +46,6 @@
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import edu.uci.ics.hyracks.storage.am.btree.api.DummySMI;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeCursor;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
@@ -68,6 +68,8 @@
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ICacheMemoryAllocator;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+import edu.uci.ics.hyracks.test.support.TestUtils;
 
 public class RangeSearchCursorTest {
     private static final int PAGE_SIZE = 256;
@@ -100,14 +102,14 @@
     IBTreeInteriorFrame interiorFrame = interiorFrameFactory.getFrame();
     IBTreeMetaDataFrame metaFrame = metaFrameFactory.getFrame();
 
-    IHyracksContext ctx = new RootHyracksContext(HYRACKS_FRAME_SIZE);
-    ByteBuffer frame = ctx.getResourceManager().allocateFrame();
-    FrameTupleAppender appender = new FrameTupleAppender(ctx);
+    IHyracksStageletContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
+    ByteBuffer frame = ctx.allocateFrame();
+    FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
 
     ISerializerDeserializer[] recDescSers = { IntegerSerializerDeserializer.INSTANCE,
             IntegerSerializerDeserializer.INSTANCE };
     RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
-    IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx, recDesc);
+    IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
     FrameTupleReference tuple = new FrameTupleReference();
 
     Random rnd = new Random(50);
@@ -123,13 +125,12 @@
     public void uniqueIndexTest() throws Exception {
 
         System.out.println("TESTING RANGE SEARCH CURSOR ON UNIQUE INDEX");
-
-        DummySMI smi = new DummySMI(PAGE_SIZE, NUM_PAGES);
-        IBufferCache bufferCache = smi.getBufferCache();
-        IFileMapProvider fmp = smi.getFileMapProvider();
-        String fileName = tmpDir + "/" + "btreetest.bin";
-        bufferCache.createFile(fileName);
-        int fileId = fmp.lookupFileId(fileName);
+        TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES);
+        IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+        IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+        FileReference file = new FileReference(new File(tmpDir + "/" + "btreetest.bin"));
+        bufferCache.createFile(file);
+        int fileId = fmp.lookupFileId(file);
         bufferCache.openFile(fileId);
 
         // declare keys
@@ -212,12 +213,12 @@
 
         System.out.println("TESTING RANGE SEARCH CURSOR ON NONUNIQUE INDEX");
 
-        DummySMI smi = new DummySMI(PAGE_SIZE, NUM_PAGES);
-        IBufferCache bufferCache = smi.getBufferCache();
-        IFileMapProvider fmp = smi.getFileMapProvider();
-        String fileName = tmpDir + "/" + "btreetest.bin";
-        bufferCache.createFile(fileName);
-        int fileId = fmp.lookupFileId(fileName);
+        TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES);
+        IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+        IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+        FileReference file = new FileReference(new File(tmpDir + "/" + "btreetest.bin"));
+        bufferCache.createFile(file);
+        int fileId = fmp.lookupFileId(file);
         bufferCache.openFile(fileId);
 
         // declare keys
@@ -301,12 +302,12 @@
         IBTreeLeafFrameFactory leafFrameFactory = new FieldPrefixNSMLeafFrameFactory(tupleWriterFactory);
         IBTreeLeafFrame leafFrame = leafFrameFactory.getFrame();
 
-        DummySMI smi = new DummySMI(PAGE_SIZE, NUM_PAGES);
-        IBufferCache bufferCache = smi.getBufferCache();
-        IFileMapProvider fmp = smi.getFileMapProvider();
-        String fileName = tmpDir + "/" + "btreetest.bin";
-        bufferCache.createFile(fileName);
-        int fileId = fmp.lookupFileId(fileName);
+        TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES);
+        IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+        IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+        FileReference file = new FileReference(new File(tmpDir + "/" + "btreetest.bin"));
+        bufferCache.createFile(file);
+        int fileId = fmp.lookupFileId(file);
         bufferCache.openFile(fileId);
 
         // declare keys
@@ -390,7 +391,7 @@
 
         ISerializerDeserializer[] keyDescSers = { IntegerSerializerDeserializer.INSTANCE };
         RecordDescriptor keyDesc = new RecordDescriptor(keyDescSers);
-        IFrameTupleAccessor keyAccessor = new FrameTupleAccessor(ctx, keyDesc);
+        IFrameTupleAccessor keyAccessor = new FrameTupleAccessor(ctx.getFrameSize(), keyDesc);
         keyAccessor.reset(frame);
 
         appender.reset(frame, true);
@@ -487,8 +488,8 @@
                     while (rangeCursor.hasNext()) {
                         rangeCursor.next();
                         ITupleReference frameTuple = rangeCursor.getTuple();
-                        ByteArrayInputStream inStream = new ByteArrayInputStream(frameTuple.getFieldData(0), frameTuple
-                                .getFieldStart(0), frameTuple.getFieldLength(0));
+                        ByteArrayInputStream inStream = new ByteArrayInputStream(frameTuple.getFieldData(0),
+                                frameTuple.getFieldStart(0), frameTuple.getFieldLength(0));
                         DataInput dataIn = new DataInputStream(inStream);
                         Integer res = IntegerSerializerDeserializer.INSTANCE.deserialize(dataIn);
                         results.add(res);
diff --git a/hyracks/hyracks-storage-am-btree/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java
similarity index 89%
rename from hyracks/hyracks-storage-am-btree/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java
rename to hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java
index b728ba0..28c360c 100644
--- a/hyracks/hyracks-storage-am-btree/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java
@@ -15,19 +15,23 @@
 
 package edu.uci.ics.hyracks.storage.am.btree;
 
+import java.io.File;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Random;
 
 import org.junit.Test;
 
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.btree.api.DummySMI;
+import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
-import edu.uci.ics.hyracks.storage.common.file.FileHandle;
+import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 import edu.uci.ics.hyracks.storage.common.sync.LatchType;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+import edu.uci.ics.hyracks.test.support.TestUtils;
 
 public class StorageManagerTest {
     private static final int PAGE_SIZE = 256;
@@ -48,7 +52,10 @@
     }
 
     public enum FileAccessType {
-        FTA_READONLY, FTA_WRITEONLY, FTA_MIXED, FTA_UNLATCHED
+        FTA_READONLY,
+        FTA_WRITEONLY,
+        FTA_MIXED,
+        FTA_UNLATCHED
     }
 
     public class FileAccessWorker implements Runnable {
@@ -85,7 +92,7 @@
             System.out.println(workerId + " PINNING PAGE: " + pageId);
 
             try {
-                ICachedPage page = bufferCache.pin(FileHandle.getDiskPageId(fileId, pageId), false);
+                ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
                 LatchType latch = null;
 
                 switch (fta) {
@@ -233,12 +240,13 @@
 
     @Test
     public void oneThreadOneFileTest() throws Exception {
-        DummySMI smi = new DummySMI(PAGE_SIZE, NUM_PAGES);
-        IBufferCache bufferCache = smi.getBufferCache();
-        IFileMapProvider fmp = smi.getFileMapProvider();
-        String fileName = tmpDir + "/" + "testfile01.bin";
-        bufferCache.createFile(fileName);
-        int fileId = fmp.lookupFileId(fileName);
+        IHyracksStageletContext ctx = TestUtils.create(32768);
+        TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES);
+        IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+        IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+        FileReference file = new FileReference(new File(tmpDir + "/" + "testfile01.bin"));
+        bufferCache.createFile(file);
+        int fileId = fmp.lookupFileId(file);
 
         Thread worker = new Thread(new FileAccessWorker(0, bufferCache, FileAccessType.FTA_UNLATCHED, fileId, 10, 10,
                 100, 10, 0));
@@ -249,5 +257,4 @@
 
         bufferCache.close();
     }
-
-}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/.classpath b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/.classpath
new file mode 100644
index 0000000..e44aa2f
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/.classpath
@@ -0,0 +1,7 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<classpath>
+	<classpathentry kind="src" path="src/test/java"/>
+	<classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/JavaSE-1.6"/>
+	<classpathentry kind="con" path="org.maven.ide.eclipse.MAVEN2_CLASSPATH_CONTAINER"/>
+	<classpathentry kind="output" path="target/classes"/>
+</classpath>
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/.project b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/.project
new file mode 100644
index 0000000..f60b2f9
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/.project
@@ -0,0 +1,23 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<projectDescription>
+	<name>hyracks-storage-am-invertedindex-test</name>
+	<comment></comment>
+	<projects>
+	</projects>
+	<buildSpec>
+		<buildCommand>
+			<name>org.eclipse.jdt.core.javabuilder</name>
+			<arguments>
+			</arguments>
+		</buildCommand>
+		<buildCommand>
+			<name>org.maven.ide.eclipse.maven2Builder</name>
+			<arguments>
+			</arguments>
+		</buildCommand>
+	</buildSpec>
+	<natures>
+		<nature>org.eclipse.jdt.core.javanature</nature>
+		<nature>org.maven.ide.eclipse.maven2Nature</nature>
+	</natures>
+</projectDescription>
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/.settings/org.eclipse.jdt.core.prefs b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/.settings/org.eclipse.jdt.core.prefs
new file mode 100644
index 0000000..3cd389e
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/.settings/org.eclipse.jdt.core.prefs
@@ -0,0 +1,6 @@
+#Thu Jan 06 11:27:16 PST 2011
+eclipse.preferences.version=1
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.6
+org.eclipse.jdt.core.compiler.compliance=1.6
+org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning
+org.eclipse.jdt.core.compiler.source=1.6
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/.settings/org.maven.ide.eclipse.prefs b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/.settings/org.maven.ide.eclipse.prefs
new file mode 100644
index 0000000..99b89a6
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/.settings/org.maven.ide.eclipse.prefs
@@ -0,0 +1,9 @@
+#Thu Jan 06 11:27:16 PST 2011
+activeProfiles=
+eclipse.preferences.version=1
+fullBuildGoals=process-test-resources
+includeModules=false
+resolveWorkspaceProjects=true
+resourceFilterGoals=process-resources resources\:testResources
+skipCompilerPlugin=true
+version=1
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/pom.xml b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/pom.xml
new file mode 100644
index 0000000..b3c62ae
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/pom.xml
@@ -0,0 +1,55 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <groupId>edu.uci.ics.hyracks</groupId>
+  <artifactId>hyracks-storage-am-invertedindex-test</artifactId>
+  <version>0.1.4-SNAPSHOT</version>
+
+  <parent>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks-tests</artifactId>
+    <version>0.1.4-SNAPSHOT</version>
+  </parent>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>2.0.2</version>
+        <configuration>
+          <source>1.6</source>
+          <target>1.6</target>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+  <dependencies>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-control-nc</artifactId>
+  		<version>0.1.4-SNAPSHOT</version>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-storage-am-invertedindex</artifactId>
+  		<version>0.1.4-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-test-support</artifactId>
+  		<version>0.1.4-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>test</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>junit</groupId>
+  		<artifactId>junit</artifactId>
+  		<version>4.8.1</version>
+  		<type>jar</type>
+  		<scope>test</scope>
+  	</dependency>
+  </dependencies>
+</project>
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchers/SimpleConjunctiveSearcherTest.java b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchers/SimpleConjunctiveSearcherTest.java
similarity index 83%
rename from hyracks/hyracks-storage-am-invertedindex/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchers/SimpleConjunctiveSearcherTest.java
rename to hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchers/SimpleConjunctiveSearcherTest.java
index 72785dd..0e53d3f 100644
--- a/hyracks/hyracks-storage-am-invertedindex/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchers/SimpleConjunctiveSearcherTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchers/SimpleConjunctiveSearcherTest.java
@@ -19,21 +19,26 @@
 import java.io.DataInput;
 import java.io.DataInputStream;
 import java.io.DataOutput;
+import java.io.File;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Random;
+import java.util.UUID;
 
 import org.junit.Test;
 
+import edu.uci.ics.hyracks.api.application.INCApplicationContext;
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
-import edu.uci.ics.hyracks.control.nc.runtime.RootHyracksContext;
+import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
@@ -42,7 +47,6 @@
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import edu.uci.ics.hyracks.storage.am.btree.api.DummySMI;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
@@ -61,12 +65,18 @@
 import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexResultCursor;
 import edu.uci.ics.hyracks.storage.am.invertedindex.impls.SimpleConjunctiveSearcher;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ICacheMemoryAllocator;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.test.support.TestJobletContext;
+import edu.uci.ics.hyracks.test.support.TestNCApplicationContext;
+import edu.uci.ics.hyracks.test.support.TestRootContext;
+import edu.uci.ics.hyracks.test.support.TestStageletContext;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
 
 public class SimpleConjunctiveSearcherTest {
-
     // testing params
     // private static final int PAGE_SIZE = 256;
     // private static final int NUM_PAGES = 10;
@@ -78,6 +88,10 @@
     private static final int NUM_PAGES = 10;
     private static final int HYRACKS_FRAME_SIZE = 32768;
 
+    static {
+        TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES);
+    }
+
     private String tmpDir = System.getProperty("java.io.tmpdir");
 
     public class BufferAllocator implements ICacheMemoryAllocator {
@@ -93,13 +107,18 @@
 
     @Test
     public void test01() throws Exception {
+        IHyracksRootContext rootCtx = new TestRootContext(HYRACKS_FRAME_SIZE);
+        INCApplicationContext appCtx = new TestNCApplicationContext(rootCtx);
+        IHyracksJobletContext jobletCtx = new TestJobletContext(appCtx, UUID.randomUUID(), 0);
+        IHyracksStageletContext stageletCtx = new TestStageletContext(jobletCtx, UUID.randomUUID());
 
-        DummySMI smi = new DummySMI(PAGE_SIZE, NUM_PAGES);
-        IBufferCache bufferCache = smi.getBufferCache();
-        IFileMapProvider fmp = smi.getFileMapProvider();
-        String fileName = tmpDir + "/" + "btreetest.bin";
-        bufferCache.createFile(fileName);
-        int fileId = fmp.lookupFileId(fileName);
+        IStorageManagerInterface smi = new TestStorageManagerInterface();
+
+        IBufferCache bufferCache = smi.getBufferCache(stageletCtx);
+        IFileMapProvider fmp = smi.getFileMapProvider(stageletCtx);
+        FileReference file = new FileReference(new File(tmpDir + "/" + "btreetest.bin"));
+        bufferCache.createFile(file);
+        int fileId = fmp.lookupFileId(file);
         bufferCache.openFile(fileId);
 
         // declare fields
@@ -136,16 +155,15 @@
         Random rnd = new Random();
         rnd.setSeed(50);
 
-        IHyracksContext ctx = new RootHyracksContext(HYRACKS_FRAME_SIZE);
-        ByteBuffer frame = ctx.getResourceManager().allocateFrame();
-        FrameTupleAppender appender = new FrameTupleAppender(ctx);
+        ByteBuffer frame = stageletCtx.allocateFrame();
+        FrameTupleAppender appender = new FrameTupleAppender(stageletCtx.getFrameSize());
         ArrayTupleBuilder tb = new ArrayTupleBuilder(cmp.getFieldCount());
         DataOutput dos = tb.getDataOutput();
 
         ISerializerDeserializer[] btreeSerde = { UTF8StringSerializerDeserializer.INSTANCE,
                 IntegerSerializerDeserializer.INSTANCE };
         RecordDescriptor btreeRecDesc = new RecordDescriptor(btreeSerde);
-        IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx, btreeRecDesc);
+        IFrameTupleAccessor accessor = new FrameTupleAccessor(stageletCtx.getFrameSize(), btreeRecDesc);
         accessor.reset(frame);
         FrameTupleReference tuple = new FrameTupleReference();
 
@@ -195,11 +213,11 @@
         ISerializerDeserializer[] querySerde = { UTF8StringSerializerDeserializer.INSTANCE };
         RecordDescriptor queryRecDesc = new RecordDescriptor(querySerde);
 
-        FrameTupleAppender queryAppender = new FrameTupleAppender(ctx);
+        FrameTupleAppender queryAppender = new FrameTupleAppender(stageletCtx.getFrameSize());
         ArrayTupleBuilder queryTb = new ArrayTupleBuilder(querySerde.length);
         DataOutput queryDos = queryTb.getDataOutput();
 
-        IFrameTupleAccessor queryAccessor = new FrameTupleAccessor(ctx, queryRecDesc);
+        IFrameTupleAccessor queryAccessor = new FrameTupleAccessor(stageletCtx.getFrameSize(), queryRecDesc);
         queryAccessor.reset(frame);
         FrameTupleReference queryTuple = new FrameTupleReference();
 
@@ -222,11 +240,11 @@
             resultSerde[i] = btreeSerde[numKeyFields + i];
         }
         RecordDescriptor resultRecDesc = new RecordDescriptor(resultSerde);
-        FrameTupleAccessor resultAccessor = new FrameTupleAccessor(ctx, resultRecDesc);
+        FrameTupleAccessor resultAccessor = new FrameTupleAccessor(stageletCtx.getFrameSize(), resultRecDesc);
         FrameTupleReference resultTuple = new FrameTupleReference();
 
-        SimpleConjunctiveSearcher searcher = new SimpleConjunctiveSearcher(ctx, btree, btreeRecDesc, queryTokenizer,
-                numKeyFields, numValueFields);
+        SimpleConjunctiveSearcher searcher = new SimpleConjunctiveSearcher(stageletCtx, btree, btreeRecDesc,
+                queryTokenizer, numKeyFields, numValueFields);
 
         long timeStart = System.currentTimeMillis();
         searcher.search(queryTuple, 0);
@@ -241,8 +259,8 @@
             for (int i = 0; i < resultAccessor.getTupleCount(); i++) {
                 resultTuple.reset(resultAccessor, i);
                 for (int j = 0; j < resultTuple.getFieldCount(); j++) {
-                    ByteArrayInputStream inStream = new ByteArrayInputStream(resultTuple.getFieldData(j), resultTuple
-                            .getFieldStart(j), resultTuple.getFieldLength(j));
+                    ByteArrayInputStream inStream = new ByteArrayInputStream(resultTuple.getFieldData(j),
+                            resultTuple.getFieldStart(j), resultTuple.getFieldLength(j));
                     DataInput dataIn = new DataInputStream(inStream);
                     Object o = resultSerde[j].deserialize(dataIn);
                     System.out.print(o + " ");
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/TokenizerTest.java b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/TokenizerTest.java
similarity index 100%
rename from hyracks/hyracks-storage-am-invertedindex/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/TokenizerTest.java
rename to hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/TokenizerTest.java
diff --git a/hyracks/hyracks-tests/pom.xml b/hyracks/hyracks-tests/pom.xml
new file mode 100644
index 0000000..47e53ae
--- /dev/null
+++ b/hyracks/hyracks-tests/pom.xml
@@ -0,0 +1,18 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <groupId>edu.uci.ics.hyracks</groupId>
+  <artifactId>hyracks-tests</artifactId>
+  <version>0.1.4-SNAPSHOT</version>
+  <packaging>pom</packaging>
+
+  <parent>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks</artifactId>
+    <version>0.1.4-SNAPSHOT</version>
+  </parent>
+
+  <modules>
+    <module>hyracks-storage-am-btree-test</module>
+    <module>hyracks-storage-am-invertedindex-test</module>
+  </modules>
+</project>
diff --git a/hyracks/pom.xml b/hyracks/pom.xml
index 5588649..0c31f34 100644
--- a/hyracks/pom.xml
+++ b/hyracks/pom.xml
@@ -65,6 +65,8 @@
     <module>hyracks-storage-common</module>
     <module>hyracks-storage-am-btree</module>
     <module>hyracks-storage-am-invertedindex</module>
+    <module>hyracks-test-support</module>
+    <module>hyracks-tests</module>
     <module>hyracks-examples</module>
     <module>hyracks-documentation</module>
   </modules>