Decouple out-of-core persistence infrastructure from out-of-core computation

Summary:
This diff proposes the following:
  - The persistence layer is decoupled from out-of-core infrastructure. This way one can simply implement different data accessors for various persistence resources. The persistence layer for reading/writing from/to local file system is implemented in this diff.
  - Previously, out-of-core data were indexed by string literals. This has changed for more flexibility. Now, data are accessible by a more flexible data indexing mechanism, in which a chain of indices are used to address a particular data.
  - With different implementations of data accessor, now there may be more emphasis on having more IO threads. It is important that these IO threads are load-balanced. In this diff, the mechanism to assign partitions to IO threads has changed.
  - All the coolness of Kryo's (de)serialization and RandomAccessFile (in D59277) is included in this diff, all at one place.

Test Plan:
mvn clean verify
out-of-core snapshot test passes

Reviewers: dionysis.logothetis, maja.kabiljo, sergey.edunov

Differential Revision: https://reviews.facebook.net/D59691
diff --git a/.gitignore b/.gitignore
index 3ae52e2..03acf8e 100644
--- a/.gitignore
+++ b/.gitignore
@@ -7,7 +7,7 @@
 # Build files:
 *.class
 target
-Unknown Job*
+UnknownJob*
 failed-profile.txt
 
 # IntelliJ IDEA files:
diff --git a/giraph-core/src/main/java/org/apache/giraph/bsp/BspService.java b/giraph-core/src/main/java/org/apache/giraph/bsp/BspService.java
index fc0fa95..9545a25 100644
--- a/giraph-core/src/main/java/org/apache/giraph/bsp/BspService.java
+++ b/giraph-core/src/main/java/org/apache/giraph/bsp/BspService.java
@@ -241,7 +241,7 @@
     this.context = context;
     this.graphTaskManager = graphTaskManager;
     this.conf = graphTaskManager.getConf();
-    this.jobId = conf.get("mapred.job.id", "Unknown Job");
+    this.jobId = conf.getJobId();
     this.taskPartition = conf.getTaskPartition();
     this.restartedSuperstep = conf.getLong(
         GiraphConstants.RESTART_SUPERSTEP, UNSET_SUPERSTEP);
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/ServerData.java b/giraph-core/src/main/java/org/apache/giraph/comm/ServerData.java
index 4156d8c..e926b6c 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/ServerData.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/ServerData.java
@@ -150,7 +150,7 @@
       oocEngine = new OutOfCoreEngine(conf, service);
       partitionStore =
           new DiskBackedPartitionStore<I, V, E>(inMemoryPartitionStore,
-              conf, context, service, oocEngine);
+              conf, context, oocEngine);
       edgeStore =
           new DiskBackedEdgeStore<I, V, E>(inMemoryEdgeStore, conf, oocEngine);
     } else {
@@ -268,7 +268,7 @@
         nextCurrentMessageStore = messageStore;
       } else {
         nextCurrentMessageStore = new DiskBackedMessageStore<>(
-            conf, messageStore,
+            conf, oocEngine, messageStore,
             conf.getIncomingMessageClasses().useMessageCombiner(),
             serviceWorker.getSuperstep());
       }
@@ -280,7 +280,7 @@
       nextIncomingMessageStore = messageStore;
     } else {
       nextIncomingMessageStore = new DiskBackedMessageStore<>(
-          conf, messageStore,
+          conf, oocEngine, messageStore,
           conf.getOutgoingMessageClasses().useMessageCombiner(),
           serviceWorker.getSuperstep() + 1);
     }
diff --git a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfiguration.java b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfiguration.java
index 7f1cb2b..4164c3a 100644
--- a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfiguration.java
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfiguration.java
@@ -1146,15 +1146,6 @@
   }
 
   /**
-   * Whether the application with change or not the graph topology.
-   *
-   * @return true if the graph is static, false otherwise.
-   */
-  public boolean isStaticGraph() {
-    return STATIC_GRAPH.isTrue(this);
-  }
-
-  /**
    * Get the output directory to write YourKit snapshots to
    *
    * @param context Map context
diff --git a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
index c592a12..ee67bed 100644
--- a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
@@ -71,8 +71,10 @@
 import org.apache.giraph.master.DefaultMasterCompute;
 import org.apache.giraph.master.MasterCompute;
 import org.apache.giraph.master.MasterObserver;
-import org.apache.giraph.ooc.OutOfCoreOracle;
-import org.apache.giraph.ooc.ThresholdBasedOracle;
+import org.apache.giraph.ooc.persistence.OutOfCoreDataAccessor;
+import org.apache.giraph.ooc.persistence.LocalDiskDataAccessor;
+import org.apache.giraph.ooc.policy.OutOfCoreOracle;
+import org.apache.giraph.ooc.policy.ThresholdBasedOracle;
 import org.apache.giraph.partition.GraphPartitionerFactory;
 import org.apache.giraph.partition.HashPartitionerFactory;
 import org.apache.giraph.partition.Partition;
@@ -949,11 +951,32 @@
           "Comma-separated list of directories in the local filesystem for " +
           "out-of-core partitions.");
 
+  /**
+   * Number of IO threads used in out-of-core mechanism. If local disk is used
+   * for spilling data to and reading data from, this number should be equal to
+   * the number of available disks on each machine. In such case, one should
+   * use giraph.partitionsDirectory to specify directories mounted on different
+   * disks.
+   */
+  IntConfOption NUM_OUT_OF_CORE_THREADS =
+      new IntConfOption("giraph.numOutOfCoreThreads", 1, "Number of IO " +
+          "threads used in out-of-core mechanism. If using local disk to " +
+          "spill data, this should be equal to the number of available " +
+          "disks. In such case, use giraph.partitionsDirectory to specify " +
+          "mount points on different disks.");
+
   /** Enable out-of-core graph. */
   BooleanConfOption USE_OUT_OF_CORE_GRAPH =
       new BooleanConfOption("giraph.useOutOfCoreGraph", false,
           "Enable out-of-core graph.");
 
+  /** Data accessor resource/object */
+  ClassConfOption<OutOfCoreDataAccessor> OUT_OF_CORE_DATA_ACCESSOR =
+      ClassConfOption.create("giraph.outOfCoreDataAccessor",
+          LocalDiskDataAccessor.class, OutOfCoreDataAccessor.class,
+          "Data accessor used in out-of-core computation (local-disk, " +
+              "in-memory, HDFS, etc.)");
+
   /**
    * Out-of-core oracle that is to be used for adaptive out-of-core engine. If
    * the `MAX_PARTITIONS_IN_MEMORY` is already set, this will be over-written
diff --git a/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfiguration.java b/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfiguration.java
index b9ecf2d..1b79cba 100644
--- a/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfiguration.java
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfiguration.java
@@ -128,6 +128,8 @@
    * extended data input/output classes for messages
    */
   private final boolean useBigDataIOForMessages;
+  /** Is the graph static (meaning there is no mutation)? */
+  private final boolean isStaticGraph;
 
   /**
    * Constructor.  Takes the configuration and then gets the classes out of
@@ -144,6 +146,7 @@
         GiraphConstants.GRAPH_TYPE_LANGUAGES, conf);
     valueNeedsWrappers = PerGraphTypeBoolean.readFromConf(
         GiraphConstants.GRAPH_TYPES_NEEDS_WRAPPERS, conf);
+    isStaticGraph = GiraphConstants.STATIC_GRAPH.get(this);
     valueFactories = new ValueFactories<I, V, E>(this);
   }
 
@@ -1326,4 +1329,20 @@
       return null;
     }
   }
+
+  /**
+   * Whether the application with change or not the graph topology.
+   *
+   * @return true if the graph is static, false otherwise.
+   */
+  public boolean isStaticGraph() {
+    return isStaticGraph;
+  }
+
+  /**
+   * @return job id
+   */
+  public String getJobId() {
+    return get("mapred.job.id", "UnknownJob");
+  }
 }
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/GraphTaskManager.java b/giraph-core/src/main/java/org/apache/giraph/graph/GraphTaskManager.java
index 725d327..a1d8522 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/GraphTaskManager.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/GraphTaskManager.java
@@ -1054,7 +1054,7 @@
    * @return Time spent in GC recorder by the GC listener
    */
   public long getSuperstepGCTime() {
-    return gcTimeMetric.count();
+    return (gcTimeMetric == null) ? 0 : gcTimeMetric.count();
   }
 
   /**
diff --git a/giraph-core/src/main/java/org/apache/giraph/ooc/OutOfCoreEngine.java b/giraph-core/src/main/java/org/apache/giraph/ooc/OutOfCoreEngine.java
index 2037abe..3187468 100644
--- a/giraph-core/src/main/java/org/apache/giraph/ooc/OutOfCoreEngine.java
+++ b/giraph-core/src/main/java/org/apache/giraph/ooc/OutOfCoreEngine.java
@@ -30,8 +30,11 @@
 import org.apache.giraph.metrics.ResetSuperstepMetricsObserver;
 import org.apache.giraph.metrics.SuperstepMetricsRegistry;
 import org.apache.giraph.ooc.data.MetaPartitionManager;
-import org.apache.giraph.ooc.io.IOCommand;
-import org.apache.giraph.ooc.io.LoadPartitionIOCommand;
+import org.apache.giraph.ooc.command.IOCommand;
+import org.apache.giraph.ooc.command.LoadPartitionIOCommand;
+import org.apache.giraph.ooc.persistence.OutOfCoreDataAccessor;
+import org.apache.giraph.ooc.policy.FixedPartitionsOracle;
+import org.apache.giraph.ooc.policy.OutOfCoreOracle;
 import org.apache.giraph.utils.AdjustableSemaphore;
 import org.apache.giraph.worker.BspServiceWorker;
 import org.apache.log4j.Logger;
@@ -87,6 +90,8 @@
    * with out-of-core operations (actual IO operations).
    */
   private final ReadWriteLock superstepLock = new ReentrantReadWriteLock();
+  /** Data accessor object (DAO) used as persistence layer in out-of-core */
+  private final OutOfCoreDataAccessor dataAccessor;
   /** Callable factory for IO threads */
   private final OutOfCoreIOCallableFactory oocIOCallableFactory;
   /**
@@ -149,9 +154,20 @@
   public OutOfCoreEngine(ImmutableClassesGiraphConfiguration<?, ?, ?> conf,
                          CentralizedServiceWorker<?, ?, ?> service) {
     this.service = service;
-    this.oocIOCallableFactory = new OutOfCoreIOCallableFactory(conf, this);
-    /* How many disk (i.e. IO threads) do we have? */
-    int numIOThreads = oocIOCallableFactory.getNumDisks();
+    Class<? extends OutOfCoreDataAccessor> accessorClass =
+        GiraphConstants.OUT_OF_CORE_DATA_ACCESSOR.get(conf);
+    try {
+      Constructor<?> constructor = accessorClass.getConstructor(
+          ImmutableClassesGiraphConfiguration.class);
+      this.dataAccessor = (OutOfCoreDataAccessor) constructor.newInstance(conf);
+    } catch (NoSuchMethodException | InstantiationException |
+        InvocationTargetException | IllegalAccessException e) {
+      throw new IllegalStateException("OutOfCoreEngine: caught exception " +
+          "while creating the data accessor instance!", e);
+    }
+    int numIOThreads = dataAccessor.getNumAccessorThreads();
+    this.oocIOCallableFactory =
+        new OutOfCoreIOCallableFactory(this, numIOThreads);
     this.ioScheduler = new OutOfCoreIOScheduler(conf, this, numIOThreads);
     this.metaPartitionManager = new MetaPartitionManager(numIOThreads, this);
     this.statistics = new OutOfCoreIOStatistics(conf, numIOThreads);
@@ -188,6 +204,7 @@
    * Initialize/Start the out-of-core engine.
    */
   public void initialize() {
+    dataAccessor.initialize();
     oocIOCallableFactory.createCallable();
   }
 
@@ -201,6 +218,7 @@
     }
     ioScheduler.shutdown();
     oocIOCallableFactory.shutdown();
+    dataAccessor.shutdown();
   }
 
   /**
@@ -500,4 +518,8 @@
   public void setFlowControl(FlowControl flowControl) {
     this.flowControl = flowControl;
   }
+
+  public OutOfCoreDataAccessor getDataAccessor() {
+    return dataAccessor;
+  }
 }
diff --git a/giraph-core/src/main/java/org/apache/giraph/ooc/OutOfCoreIOCallable.java b/giraph-core/src/main/java/org/apache/giraph/ooc/OutOfCoreIOCallable.java
index 962bd6a..bea3994 100644
--- a/giraph-core/src/main/java/org/apache/giraph/ooc/OutOfCoreIOCallable.java
+++ b/giraph-core/src/main/java/org/apache/giraph/ooc/OutOfCoreIOCallable.java
@@ -23,9 +23,9 @@
 import org.apache.giraph.metrics.GiraphMetrics;
 import org.apache.giraph.metrics.ResetSuperstepMetricsObserver;
 import org.apache.giraph.metrics.SuperstepMetricsRegistry;
-import org.apache.giraph.ooc.io.IOCommand;
-import org.apache.giraph.ooc.io.LoadPartitionIOCommand;
-import org.apache.giraph.ooc.io.WaitIOCommand;
+import org.apache.giraph.ooc.command.IOCommand;
+import org.apache.giraph.ooc.command.LoadPartitionIOCommand;
+import org.apache.giraph.ooc.command.WaitIOCommand;
 import org.apache.log4j.Logger;
 
 import java.util.concurrent.Callable;
@@ -47,8 +47,6 @@
   private static final Logger LOG = Logger.getLogger(OutOfCoreIOCallable.class);
   /** Out-of-core engine */
   private final OutOfCoreEngine oocEngine;
-  /** Base path that this thread will write to/read from */
-  private final String basePath;
   /** Thread id/Disk id */
   private final int diskId;
   /** How many bytes of data is read from disk */
@@ -64,13 +62,10 @@
    * Constructor
    *
    * @param oocEngine out-of-core engine
-   * @param basePath base path this thread will be using
    * @param diskId thread id/disk id
    */
-  public OutOfCoreIOCallable(OutOfCoreEngine oocEngine, String basePath,
-                             int diskId) {
+  public OutOfCoreIOCallable(OutOfCoreEngine oocEngine, int diskId) {
     this.oocEngine = oocEngine;
-    this.basePath = basePath;
     this.diskId = diskId;
     newSuperstep(GiraphMetrics.get().perSuperstep());
     GiraphMetrics.get().addSuperstepResetObserver(this);
@@ -98,15 +93,23 @@
       long bytes;
       // CHECKSTYLE: stop IllegalCatch
       try {
+        long timeInGC = oocEngine.getServiceWorker().getGraphTaskManager()
+            .getSuperstepGCTime();
         long startTime = System.currentTimeMillis();
-        commandExecuted = command.execute(basePath);
+        commandExecuted = command.execute();
         duration = System.currentTimeMillis() - startTime;
+        timeInGC = oocEngine.getServiceWorker().getGraphTaskManager()
+            .getSuperstepGCTime() - timeInGC;
         bytes = command.bytesTransferred();
         if (LOG.isInfoEnabled()) {
           LOG.info("call: thread " + diskId + "'s command " + command +
               " completed: bytes= " + bytes + ", duration=" + duration + ", " +
               "bandwidth=" + String.format("%.2f", (double) bytes / duration *
-              1000 / 1024 / 1024));
+              1000 / 1024 / 1024) +
+              ((command instanceof WaitIOCommand) ? "" :
+                  (", bandwidth (excluding GC time)=" + String.format("%.2f",
+                      (double) bytes / (duration - timeInGC) *
+                          1000 / 1024 / 1024))));
         }
       } catch (Exception e) {
         oocEngine.failTheJob();
diff --git a/giraph-core/src/main/java/org/apache/giraph/ooc/OutOfCoreIOCallableFactory.java b/giraph-core/src/main/java/org/apache/giraph/ooc/OutOfCoreIOCallableFactory.java
index d4fea22..6aeb196 100644
--- a/giraph-core/src/main/java/org/apache/giraph/ooc/OutOfCoreIOCallableFactory.java
+++ b/giraph-core/src/main/java/org/apache/giraph/ooc/OutOfCoreIOCallableFactory.java
@@ -18,13 +18,11 @@
 
 package org.apache.giraph.ooc;
 
-import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.utils.CallableFactory;
 import org.apache.giraph.utils.LogStacktraceCallable;
 import org.apache.giraph.utils.ThreadUtils;
 import org.apache.log4j.Logger;
 
-import java.io.File;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.Callable;
@@ -35,9 +33,6 @@
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
-import static com.google.common.base.Preconditions.checkState;
-import static org.apache.giraph.conf.GiraphConstants.PARTITIONS_DIRECTORY;
-
 /**
  * Factory class to create IO threads for out-of-core engine.
  */
@@ -49,37 +44,22 @@
   private final OutOfCoreEngine oocEngine;
   /** Result of IO threads at the end of the computation */
   private final List<Future> results;
-  /** How many disks (i.e. IO threads) do we have? */
-  private int numDisks;
-  /** Path prefix for different disks */
-  private final String[] basePaths;
+  /** Number of threads used for IO operations */
+  private final int numIOThreads;
   /** Executor service for IO threads */
   private ExecutorService outOfCoreIOExecutor;
+
   /**
    * Constructor
    *
-   * @param conf Configuration
    * @param oocEngine Out-of-core engine
+   * @param numIOThreads Number of IO threads used
    */
-  public OutOfCoreIOCallableFactory(
-      ImmutableClassesGiraphConfiguration<?, ?, ?> conf,
-      OutOfCoreEngine oocEngine) {
+  public OutOfCoreIOCallableFactory(OutOfCoreEngine oocEngine,
+                                    int numIOThreads) {
     this.oocEngine = oocEngine;
-    this.results = new ArrayList<>();
-    // Take advantage of multiple disks
-    String[] userPaths = PARTITIONS_DIRECTORY.getArray(conf);
-    this.numDisks = userPaths.length;
-    this.basePaths = new String[numDisks];
-    int ptr = 0;
-    for (String path : userPaths) {
-      File file = new File(path);
-      if (!file.exists()) {
-        checkState(file.mkdirs(), "OutOfCoreIOCallableFactory: cannot create " +
-            "directory " + file.getAbsolutePath());
-      }
-      basePaths[ptr] = path + "/" + conf.get("mapred.job.id", "Unknown Job");
-      ptr++;
-    }
+    this.numIOThreads = numIOThreads;
+    this.results = new ArrayList<>(numIOThreads);
   }
 
   /**
@@ -90,11 +70,10 @@
       new CallableFactory<Void>() {
         @Override
         public Callable<Void> newCallable(int callableId) {
-          return new OutOfCoreIOCallable(oocEngine, basePaths[callableId],
-              callableId);
+          return new OutOfCoreIOCallable(oocEngine, callableId);
         }
       };
-    outOfCoreIOExecutor = new ThreadPoolExecutor(numDisks, numDisks, 0L,
+    outOfCoreIOExecutor = new ThreadPoolExecutor(numIOThreads, numIOThreads, 0L,
         TimeUnit.MILLISECONDS, new LinkedBlockingQueue<Runnable>(),
         ThreadUtils.createThreadFactory("ooc-io-%d")) {
       @Override
@@ -120,7 +99,7 @@
       }
     };
 
-    for (int i = 0; i < numDisks; ++i) {
+    for (int i = 0; i < numIOThreads; ++i) {
       Future<Void> future = outOfCoreIOExecutor.submit(
           new LogStacktraceCallable<>(
               outOfCoreIOCallableFactory.newCallable(i)));
@@ -131,15 +110,6 @@
   }
 
   /**
-   * How many disks do we have?
-   *
-   * @return number of disks (IO threads)
-   */
-  public int getNumDisks() {
-    return numDisks;
-  }
-
-  /**
    * Check whether all IO threads terminated gracefully.
    */
   public void shutdown() {
@@ -156,7 +126,7 @@
             "InterruptedException while waiting for IO threads to finish");
       }
     }
-    for (int i = 0; i < numDisks; ++i) {
+    for (int i = 0; i < numIOThreads; ++i) {
       try {
         // Check whether the tread terminated gracefully
         results.get(i).get();
@@ -170,15 +140,5 @@
         throw new IllegalStateException(e);
       }
     }
-    for (String path : basePaths) {
-      File file = new File(path).getParentFile();
-      for (String subFileName : file.list()) {
-        File subFile = new File(file.getPath(), subFileName);
-        checkState(subFile.delete(), "shutdown: cannot delete file %s",
-            subFile.getAbsoluteFile());
-      }
-      checkState(file.delete(), "shutdown: cannot delete directory %s",
-          file.getAbsoluteFile());
-    }
   }
 }
diff --git a/giraph-core/src/main/java/org/apache/giraph/ooc/OutOfCoreIOScheduler.java b/giraph-core/src/main/java/org/apache/giraph/ooc/OutOfCoreIOScheduler.java
index 6428c30..906607d 100644
--- a/giraph-core/src/main/java/org/apache/giraph/ooc/OutOfCoreIOScheduler.java
+++ b/giraph-core/src/main/java/org/apache/giraph/ooc/OutOfCoreIOScheduler.java
@@ -18,15 +18,15 @@
 
 package org.apache.giraph.ooc;
 
-import com.google.common.hash.Hashing;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.conf.IntConfOption;
-import org.apache.giraph.ooc.io.IOCommand;
-import org.apache.giraph.ooc.io.LoadPartitionIOCommand;
-import org.apache.giraph.ooc.io.StoreDataBufferIOCommand;
-import org.apache.giraph.ooc.io.StoreIncomingMessageIOCommand;
-import org.apache.giraph.ooc.io.StorePartitionIOCommand;
-import org.apache.giraph.ooc.io.WaitIOCommand;
+import org.apache.giraph.ooc.command.IOCommand;
+import org.apache.giraph.ooc.command.LoadPartitionIOCommand;
+import org.apache.giraph.ooc.command.StoreDataBufferIOCommand;
+import org.apache.giraph.ooc.command.StoreIncomingMessageIOCommand;
+import org.apache.giraph.ooc.command.StorePartitionIOCommand;
+import org.apache.giraph.ooc.command.WaitIOCommand;
+import org.apache.giraph.ooc.policy.OutOfCoreOracle;
 import org.apache.log4j.Logger;
 
 import java.util.ArrayList;
@@ -57,8 +57,6 @@
   private final OutOfCoreEngine oocEngine;
   /** How much an IO thread should wait if there is no IO command */
   private final int waitInterval;
-  /** How many disks (i.e. IO threads) do we have? */
-  private final int numDisks;
   /**
    * Queue of IO commands for loading partitions to memory. Load commands are
    * urgent and should be done once loading data is a viable IO command.
@@ -77,7 +75,6 @@
   OutOfCoreIOScheduler(final ImmutableClassesGiraphConfiguration conf,
                        OutOfCoreEngine oocEngine, int numDisks) {
     this.oocEngine = oocEngine;
-    this.numDisks = numDisks;
     this.waitInterval = OOC_WAIT_INTERVAL.get(conf);
     threadLoadCommandQueue = new ArrayList<>(numDisks);
     for (int i = 0; i < numDisks; ++i) {
@@ -88,17 +85,6 @@
   }
 
   /**
-   * Get the thread id that is responsible for a particular partition
-   *
-   * @param partitionId id of the given partition
-   * @return id of the thread responsible for the given partition
-   */
-  public int getOwnerThreadId(int partitionId) {
-    int result = Hashing.murmur3_32().hashInt(partitionId).asInt() % numDisks;
-    return (result >= 0) ? result : (result + numDisks);
-  }
-
-  /**
    * Generate and return the next appropriate IO command for a given thread
    *
    * @param threadId id of the thread ready to execute the next IO command
@@ -254,8 +240,9 @@
    * @param ioCommand IO command to add to the scheduler
    */
   public void addIOCommand(IOCommand ioCommand) {
-    int ownerThread = getOwnerThreadId(ioCommand.getPartitionId());
     if (ioCommand instanceof LoadPartitionIOCommand) {
+      int ownerThread = oocEngine.getMetaPartitionManager()
+          .getOwnerThreadId(ioCommand.getPartitionId());
       threadLoadCommandQueue.get(ownerThread).offer(ioCommand);
     } else {
       throw new IllegalStateException("addIOCommand: IO command type is not " +
diff --git a/giraph-core/src/main/java/org/apache/giraph/ooc/OutOfCoreIOStatistics.java b/giraph-core/src/main/java/org/apache/giraph/ooc/OutOfCoreIOStatistics.java
index a225a4c..44a0d2f 100644
--- a/giraph-core/src/main/java/org/apache/giraph/ooc/OutOfCoreIOStatistics.java
+++ b/giraph-core/src/main/java/org/apache/giraph/ooc/OutOfCoreIOStatistics.java
@@ -22,7 +22,7 @@
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.conf.IntConfOption;
-import org.apache.giraph.ooc.io.IOCommand.IOCommandType;
+import org.apache.giraph.ooc.command.IOCommand.IOCommandType;
 import org.apache.log4j.Logger;
 
 import java.util.Map;
diff --git a/giraph-core/src/main/java/org/apache/giraph/ooc/io/IOCommand.java b/giraph-core/src/main/java/org/apache/giraph/ooc/command/IOCommand.java
similarity index 92%
rename from giraph-core/src/main/java/org/apache/giraph/ooc/io/IOCommand.java
rename to giraph-core/src/main/java/org/apache/giraph/ooc/command/IOCommand.java
index e84ad29..b6c986d 100644
--- a/giraph-core/src/main/java/org/apache/giraph/ooc/io/IOCommand.java
+++ b/giraph-core/src/main/java/org/apache/giraph/ooc/command/IOCommand.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.giraph.ooc.io;
+package org.apache.giraph.ooc.command;
 
 import org.apache.giraph.ooc.OutOfCoreEngine;
 
@@ -80,12 +80,10 @@
    * appropriately based on the data loaded/stored. Return true iff the command
    * is actually executed (resulted in loading or storing data).
    *
-   * @param basePath the base path (prefix) to the files/folders IO command
-   *                 should read/write data from/to
    * @return whether the command is actually executed
    * @throws IOException
    */
-  public abstract boolean execute(String basePath) throws IOException;
+  public abstract boolean execute() throws IOException;
 
   /**
    * Get the type of the command.
diff --git a/giraph-core/src/main/java/org/apache/giraph/ooc/io/LoadPartitionIOCommand.java b/giraph-core/src/main/java/org/apache/giraph/ooc/command/LoadPartitionIOCommand.java
similarity index 92%
rename from giraph-core/src/main/java/org/apache/giraph/ooc/io/LoadPartitionIOCommand.java
rename to giraph-core/src/main/java/org/apache/giraph/ooc/command/LoadPartitionIOCommand.java
index ce24fe2..ee12159 100644
--- a/giraph-core/src/main/java/org/apache/giraph/ooc/io/LoadPartitionIOCommand.java
+++ b/giraph-core/src/main/java/org/apache/giraph/ooc/command/LoadPartitionIOCommand.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.giraph.ooc.io;
+package org.apache.giraph.ooc.command;
 
 import com.google.common.base.Preconditions;
 import org.apache.giraph.bsp.BspService;
@@ -54,7 +54,7 @@
   }
 
   @Override
-  public boolean execute(String basePath) throws IOException {
+  public boolean execute() throws IOException {
     boolean executed = false;
     if (oocEngine.getMetaPartitionManager()
         .startLoadingPartition(partitionId, superstep)) {
@@ -63,13 +63,13 @@
           (DiskBackedPartitionStore)
               oocEngine.getServerData().getPartitionStore();
       numBytesTransferred +=
-          partitionStore.loadPartitionData(partitionId, basePath);
+          partitionStore.loadPartitionData(partitionId);
       if (currentSuperstep == BspService.INPUT_SUPERSTEP &&
           superstep == currentSuperstep) {
         DiskBackedEdgeStore edgeStore =
             (DiskBackedEdgeStore) oocEngine.getServerData().getEdgeStore();
         numBytesTransferred +=
-            edgeStore.loadPartitionData(partitionId, basePath);
+            edgeStore.loadPartitionData(partitionId);
       }
       MessageStore messageStore;
       if (currentSuperstep == superstep) {
@@ -80,7 +80,7 @@
       }
       if (messageStore != null) {
         numBytesTransferred += ((DiskBackedMessageStore) messageStore)
-            .loadPartitionData(partitionId, basePath);
+            .loadPartitionData(partitionId);
       }
       oocEngine.getMetaPartitionManager()
           .doneLoadingPartition(partitionId, superstep);
diff --git a/giraph-core/src/main/java/org/apache/giraph/ooc/io/StoreDataBufferIOCommand.java b/giraph-core/src/main/java/org/apache/giraph/ooc/command/StoreDataBufferIOCommand.java
similarity index 92%
rename from giraph-core/src/main/java/org/apache/giraph/ooc/io/StoreDataBufferIOCommand.java
rename to giraph-core/src/main/java/org/apache/giraph/ooc/command/StoreDataBufferIOCommand.java
index f1769dd..beda796 100644
--- a/giraph-core/src/main/java/org/apache/giraph/ooc/io/StoreDataBufferIOCommand.java
+++ b/giraph-core/src/main/java/org/apache/giraph/ooc/command/StoreDataBufferIOCommand.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.giraph.ooc.io;
+package org.apache.giraph.ooc.command;
 
 import org.apache.giraph.ooc.OutOfCoreEngine;
 import org.apache.giraph.ooc.data.DiskBackedEdgeStore;
@@ -54,7 +54,7 @@
   }
 
   @Override
-  public boolean execute(String basePath) throws IOException {
+  public boolean execute() throws IOException {
     boolean executed = false;
     if (oocEngine.getMetaPartitionManager()
         .startOffloadingBuffer(partitionId)) {
@@ -64,17 +64,17 @@
             (DiskBackedPartitionStore)
                 oocEngine.getServerData().getPartitionStore();
         numBytesTransferred +=
-            partitionStore.offloadBuffers(partitionId, basePath);
+            partitionStore.offloadBuffers(partitionId);
         DiskBackedEdgeStore edgeStore =
             (DiskBackedEdgeStore) oocEngine.getServerData().getEdgeStore();
-        numBytesTransferred += edgeStore.offloadBuffers(partitionId, basePath);
+        numBytesTransferred += edgeStore.offloadBuffers(partitionId);
         break;
       case MESSAGE:
         DiskBackedMessageStore messageStore =
             (DiskBackedMessageStore)
                 oocEngine.getServerData().getIncomingMessageStore();
         numBytesTransferred +=
-            messageStore.offloadBuffers(partitionId, basePath);
+            messageStore.offloadBuffers(partitionId);
         break;
       default:
         throw new IllegalStateException("execute: requested data buffer type " +
diff --git a/giraph-core/src/main/java/org/apache/giraph/ooc/io/StoreIncomingMessageIOCommand.java b/giraph-core/src/main/java/org/apache/giraph/ooc/command/StoreIncomingMessageIOCommand.java
similarity index 92%
rename from giraph-core/src/main/java/org/apache/giraph/ooc/io/StoreIncomingMessageIOCommand.java
rename to giraph-core/src/main/java/org/apache/giraph/ooc/command/StoreIncomingMessageIOCommand.java
index c9d8829..b38f957 100644
--- a/giraph-core/src/main/java/org/apache/giraph/ooc/io/StoreIncomingMessageIOCommand.java
+++ b/giraph-core/src/main/java/org/apache/giraph/ooc/command/StoreIncomingMessageIOCommand.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.giraph.ooc.io;
+package org.apache.giraph.ooc.command;
 
 import org.apache.giraph.ooc.OutOfCoreEngine;
 import org.apache.giraph.ooc.data.DiskBackedMessageStore;
@@ -41,7 +41,7 @@
   }
 
   @Override
-  public boolean execute(String basePath) throws IOException {
+  public boolean execute() throws IOException {
     boolean executed = false;
     if (oocEngine.getMetaPartitionManager()
         .startOffloadingMessages(partitionId)) {
@@ -50,7 +50,7 @@
               oocEngine.getServerData().getIncomingMessageStore();
       checkState(messageStore != null);
       numBytesTransferred +=
-          messageStore.offloadPartitionData(partitionId, basePath);
+          messageStore.offloadPartitionData(partitionId);
       oocEngine.getMetaPartitionManager().doneOffloadingMessages(partitionId);
       executed = true;
     }
diff --git a/giraph-core/src/main/java/org/apache/giraph/ooc/io/StorePartitionIOCommand.java b/giraph-core/src/main/java/org/apache/giraph/ooc/command/StorePartitionIOCommand.java
similarity index 90%
rename from giraph-core/src/main/java/org/apache/giraph/ooc/io/StorePartitionIOCommand.java
rename to giraph-core/src/main/java/org/apache/giraph/ooc/command/StorePartitionIOCommand.java
index 797ac9d..31fa345 100644
--- a/giraph-core/src/main/java/org/apache/giraph/ooc/io/StorePartitionIOCommand.java
+++ b/giraph-core/src/main/java/org/apache/giraph/ooc/command/StorePartitionIOCommand.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.giraph.ooc.io;
+package org.apache.giraph.ooc.command;
 
 import org.apache.giraph.bsp.BspService;
 import org.apache.giraph.comm.messages.MessageStore;
@@ -44,7 +44,7 @@
   }
 
   @Override
-  public boolean execute(String basePath) throws IOException {
+  public boolean execute() throws IOException {
     boolean executed = false;
     if (oocEngine.getMetaPartitionManager()
         .startOffloadingPartition(partitionId)) {
@@ -52,20 +52,20 @@
           (DiskBackedPartitionStore)
               oocEngine.getServerData().getPartitionStore();
       numBytesTransferred +=
-          partitionStore.offloadPartitionData(partitionId, basePath);
+          partitionStore.offloadPartitionData(partitionId);
       if (oocEngine.getSuperstep() != BspService.INPUT_SUPERSTEP) {
         MessageStore messageStore =
             oocEngine.getServerData().getCurrentMessageStore();
         if (messageStore != null) {
           numBytesTransferred += ((DiskBackedMessageStore) messageStore)
-              .offloadPartitionData(partitionId, basePath);
+              .offloadPartitionData(partitionId);
         }
       } else {
         DiskBackedEdgeStore edgeStore =
             (DiskBackedEdgeStore)
                 oocEngine.getServerData().getEdgeStore();
         numBytesTransferred +=
-            edgeStore.offloadPartitionData(partitionId, basePath);
+            edgeStore.offloadPartitionData(partitionId);
       }
       oocEngine.getMetaPartitionManager().doneOffloadingPartition(partitionId);
       executed = true;
diff --git a/giraph-core/src/main/java/org/apache/giraph/ooc/io/WaitIOCommand.java b/giraph-core/src/main/java/org/apache/giraph/ooc/command/WaitIOCommand.java
similarity index 94%
rename from giraph-core/src/main/java/org/apache/giraph/ooc/io/WaitIOCommand.java
rename to giraph-core/src/main/java/org/apache/giraph/ooc/command/WaitIOCommand.java
index 74e72eb..83540c1 100644
--- a/giraph-core/src/main/java/org/apache/giraph/ooc/io/WaitIOCommand.java
+++ b/giraph-core/src/main/java/org/apache/giraph/ooc/command/WaitIOCommand.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.giraph.ooc.io;
+package org.apache.giraph.ooc.command;
 
 import org.apache.giraph.ooc.OutOfCoreEngine;
 
@@ -42,7 +42,7 @@
   }
 
   @Override
-  public boolean execute(String basePath) throws IOException {
+  public boolean execute() throws IOException {
     try {
       TimeUnit.MILLISECONDS.sleep(waitDuration);
     } catch (InterruptedException e) {
diff --git a/giraph-core/src/main/java/org/apache/giraph/ooc/io/package-info.java b/giraph-core/src/main/java/org/apache/giraph/ooc/command/package-info.java
similarity index 88%
copy from giraph-core/src/main/java/org/apache/giraph/ooc/io/package-info.java
copy to giraph-core/src/main/java/org/apache/giraph/ooc/command/package-info.java
index 2230ec4..930b139 100644
--- a/giraph-core/src/main/java/org/apache/giraph/ooc/io/package-info.java
+++ b/giraph-core/src/main/java/org/apache/giraph/ooc/command/package-info.java
@@ -16,6 +16,6 @@
  * limitations under the License.
  */
 /**
- * Package of classes related to IO operations in out-of-core mechanism
+ * Package of classes related to IO commands in out-of-core mechanism
  */
-package org.apache.giraph.ooc.io;
+package org.apache.giraph.ooc.command;
diff --git a/giraph-core/src/main/java/org/apache/giraph/ooc/data/OutOfCoreDataManager.java b/giraph-core/src/main/java/org/apache/giraph/ooc/data/DiskBackedDataStore.java
similarity index 68%
rename from giraph-core/src/main/java/org/apache/giraph/ooc/data/OutOfCoreDataManager.java
rename to giraph-core/src/main/java/org/apache/giraph/ooc/data/DiskBackedDataStore.java
index 325850c..7265410 100644
--- a/giraph-core/src/main/java/org/apache/giraph/ooc/data/OutOfCoreDataManager.java
+++ b/giraph-core/src/main/java/org/apache/giraph/ooc/data/DiskBackedDataStore.java
@@ -24,17 +24,14 @@
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.conf.IntConfOption;
+import org.apache.giraph.ooc.OutOfCoreEngine;
+import org.apache.giraph.ooc.persistence.DataIndex;
+import org.apache.giraph.ooc.persistence.DataIndex.NumericIndexEntry;
+import org.apache.giraph.ooc.persistence.OutOfCoreDataAccessor;
 import org.apache.log4j.Logger;
 
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
 import java.io.DataInput;
-import java.io.DataInputStream;
 import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashSet;
@@ -71,7 +68,7 @@
  *
  * @param <T> raw data format of the data store subclassing this class
  */
-public abstract class OutOfCoreDataManager<T> {
+public abstract class DiskBackedDataStore<T> {
   /**
    * Minimum size of a buffer (in bytes) to flush to disk. This is used to
    * decide whether vertex/edge buffers are large enough to flush to disk.
@@ -82,7 +79,30 @@
 
   /** Class logger. */
   private static final Logger LOG = Logger.getLogger(
-      OutOfCoreDataManager.class);
+      DiskBackedDataStore.class);
+  /** Out-of-core engine */
+  protected final OutOfCoreEngine oocEngine;
+  /**
+   * Set containing ids of all partitions where the partition data is in some
+   * file on disk.
+   * Note that the out-of-core mechanism may decide to put the data for a
+   * partition on disk, while the partition data is empty. For instance, at the
+   * beginning of a superstep, out-of-core mechanism may decide to put incoming
+   * messages of a partition on disk, while the partition has not received any
+   * messages. In such scenarios, the "out-of-core mechanism" thinks that the
+   * partition data is on disk, while disk-backed data stores may want to
+   * optimize for IO/metadata accesses and decide not to create/write anything
+   * on files on disk.
+   * In summary, there is a subtle difference between this field and
+   * `hasPartitionOnDisk` field. Basically, this field is used for optimizing
+   * IO (mainly metadata) accesses by disk-backed stores, while
+   * `hasPartitionDataOnDisk` is the view that out-of-core mechanism has
+   * regarding partition storage statuses. Since out-of-core mechanism does not
+   * know about the actual data for a partition, these two fields have to be
+   * separate.
+   */
+  protected final Set<Integer> hasPartitionDataOnFile =
+      Sets.newConcurrentHashSet();
   /** Cached value for MINIMUM_BUFFER_SIZE_TO_FLUSH */
   private final int minBufferSizeToOffload;
   /** Set containing ids of all out-of-core partitions */
@@ -117,9 +137,12 @@
    * Constructor.
    *
    * @param conf Configuration
+   * @param oocEngine Out-of-core engine
    */
-  OutOfCoreDataManager(ImmutableClassesGiraphConfiguration conf) {
+  DiskBackedDataStore(ImmutableClassesGiraphConfiguration conf,
+                      OutOfCoreEngine oocEngine) {
     this.minBufferSizeToOffload = MINIMUM_BUFFER_SIZE_TO_FLUSH.get(conf);
+    this.oocEngine = oocEngine;
   }
 
   /**
@@ -174,7 +197,7 @@
         }
       }
     } else {
-      addEntryToImMemoryPartitionData(partitionId, entry);
+      addEntryToInMemoryPartitionData(partitionId, entry);
     }
     rwLock.readLock().unlock();
   }
@@ -184,48 +207,53 @@
    * data store. Returns the number of bytes transferred from disk to memory in
    * the loading process.
    *
-   * @param partitionId id of the partition to load ana assemble all data for
-   * @param basePath path to load the data from
+   * @param partitionId id of the partition to load and assemble all data for
    * @return number of bytes loaded from disk to memory
    * @throws IOException
    */
-  public long loadPartitionData(int partitionId, String basePath)
+  public abstract long loadPartitionData(int partitionId) throws IOException;
+
+  /**
+   * The proxy method that does the actual operation for `loadPartitionData`,
+   * but uses the data index given by the caller.
+   *
+   * @param partitionId id of the partition to load and assemble all data for
+   * @param index data index chain for the data to load
+   * @return number of bytes loaded from disk to memory
+   * @throws IOException
+   */
+  protected long loadPartitionDataProxy(int partitionId, DataIndex index)
       throws IOException {
     long numBytes = 0;
     ReadWriteLock rwLock = getPartitionLock(partitionId);
     rwLock.writeLock().lock();
     if (hasPartitionDataOnDisk.contains(partitionId)) {
-      numBytes += loadInMemoryPartitionData(partitionId,
-          getPath(basePath, partitionId));
+      int ioThreadId =
+          oocEngine.getMetaPartitionManager().getOwnerThreadId(partitionId);
+      numBytes += loadInMemoryPartitionData(partitionId, ioThreadId,
+          index.addIndex(NumericIndexEntry.createPartitionEntry(partitionId)));
       hasPartitionDataOnDisk.remove(partitionId);
       // Loading raw data buffers from disk if there is any and applying those
       // to already loaded in-memory data.
       Integer numBuffers = numDataBuffersOnDisk.remove(partitionId);
       if (numBuffers != null) {
         checkState(numBuffers > 0);
-        File file = new File(getBuffersPath(basePath, partitionId));
-        checkState(file.exists());
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("loadPartitionData: loading " + numBuffers + " buffers of" +
-              " partition " + partitionId + " from " + file.getAbsolutePath());
-        }
-        FileInputStream fis = new FileInputStream(file);
-        BufferedInputStream bis = new BufferedInputStream(fis);
-        DataInputStream dis = new DataInputStream(bis);
+        index.addIndex(DataIndex.TypeIndexEntry.BUFFER);
+        OutOfCoreDataAccessor.DataInputWrapper inputWrapper =
+            oocEngine.getDataAccessor().prepareInput(ioThreadId, index.copy());
         for (int i = 0; i < numBuffers; ++i) {
-          T entry = readNextEntry(dis);
-          addEntryToImMemoryPartitionData(partitionId, entry);
+          T entry = readNextEntry(inputWrapper.getDataInput());
+          addEntryToInMemoryPartitionData(partitionId, entry);
         }
-        dis.close();
-        numBytes +=  file.length();
-        checkState(file.delete(), "loadPartitionData: failed to delete %s.",
-            file.getAbsoluteFile());
+        numBytes += inputWrapper.finalizeInput(true);
+        index.removeLastIndex();
       }
+      index.removeLastIndex();
       // Applying in-memory raw data buffers to in-memory partition data.
       Pair<Integer, List<T>> pair = dataBuffers.remove(partitionId);
       if (pair != null) {
         for (T entry : pair.getValue()) {
-          addEntryToImMemoryPartitionData(partitionId, entry);
+          addEntryToInMemoryPartitionData(partitionId, entry);
         }
       }
     }
@@ -238,20 +266,34 @@
    * returns the number of bytes offloaded from memory to disk.
    *
    * @param partitionId id of the partition to offload its data
-   * @param basePath path to offload the data to
+   * @return number of bytes offloaded from memory to disk
+   * @throws IOException
+   */
+  public abstract long offloadPartitionData(int partitionId) throws IOException;
+
+  /**
+   * The proxy method that does the actual operation for `offloadPartitionData`,
+   * but uses the data index given by the caller.
+   *
+   * @param partitionId id of the partition to offload its data
+   * @param index data index chain for the data to offload
    * @return number of bytes offloaded from memory to disk
    * @throws IOException
    */
   @edu.umd.cs.findbugs.annotations.SuppressWarnings(
       "UL_UNRELEASED_LOCK_EXCEPTION_PATH")
-  public long offloadPartitionData(int partitionId, String basePath)
-      throws IOException {
+  protected long offloadPartitionDataProxy(
+      int partitionId, DataIndex index) throws IOException {
     ReadWriteLock rwLock = getPartitionLock(partitionId);
     rwLock.writeLock().lock();
     hasPartitionDataOnDisk.add(partitionId);
     rwLock.writeLock().unlock();
-    return offloadInMemoryPartitionData(partitionId,
-        getPath(basePath, partitionId));
+    int ioThreadId =
+        oocEngine.getMetaPartitionManager().getOwnerThreadId(partitionId);
+    long numBytes = offloadInMemoryPartitionData(partitionId, ioThreadId,
+        index.addIndex(NumericIndexEntry.createPartitionEntry(partitionId)));
+    index.removeLastIndex();
+    return numBytes;
   }
 
   /**
@@ -259,11 +301,21 @@
    * number of bytes offloaded from memory to disk.
    *
    * @param partitionId id of the partition to offload its raw data buffers
-   * @param basePath path to offload the data to
    * @return number of bytes offloaded from memory to disk
    * @throws IOException
    */
-  public long offloadBuffers(int partitionId, String basePath)
+  public abstract long offloadBuffers(int partitionId) throws IOException;
+
+  /**
+   * The proxy method that does the actual operation for `offloadBuffers`,
+   * but uses the data index given by the caller.
+   *
+   * @param partitionId id of the partition to offload its raw data buffers
+   * @param index data index chain for the data to offload its buffers
+   * @return number of bytes offloaded from memory to disk
+   * @throws IOException
+   */
+  protected long offloadBuffersProxy(int partitionId, DataIndex index)
       throws IOException {
     Pair<Integer, List<T>> pair = dataBuffers.get(partitionId);
     if (pair == null || pair.getLeft() < minBufferSizeToOffload) {
@@ -275,15 +327,18 @@
     rwLock.writeLock().unlock();
     checkNotNull(pair);
     checkState(!pair.getRight().isEmpty());
-    File file = new File(getBuffersPath(basePath, partitionId));
-    FileOutputStream fos = new FileOutputStream(file, true);
-    BufferedOutputStream bos = new BufferedOutputStream(fos);
-    DataOutputStream dos = new DataOutputStream(bos);
+    int ioThreadId =
+        oocEngine.getMetaPartitionManager().getOwnerThreadId(partitionId);
+    index.addIndex(NumericIndexEntry.createPartitionEntry(partitionId))
+        .addIndex(DataIndex.TypeIndexEntry.BUFFER);
+    OutOfCoreDataAccessor.DataOutputWrapper outputWrapper =
+        oocEngine.getDataAccessor().prepareOutput(ioThreadId, index.copy(),
+            true);
     for (T entry : pair.getRight()) {
-      writeEntry(entry, dos);
+      writeEntry(entry, outputWrapper.getDataOutput());
     }
-    dos.close();
-    long numBytes = dos.size();
+    long numBytes = outputWrapper.finalizeOutput();
+    index.removeLastIndex().removeLastIndex();
     int numBuffers = pair.getRight().size();
     Integer oldNumBuffersOnDisk =
         numDataBuffersOnDisk.putIfAbsent(partitionId, numBuffers);
@@ -315,30 +370,6 @@
   }
 
   /**
-   * Creates the path to read/write partition data from/to for a given
-   * partition.
-   *
-   * @param basePath path prefix to create the actual path from
-   * @param partitionId id of the partition
-   * @return path to read/write data from/to
-   */
-  private static String getPath(String basePath, int partitionId) {
-    return basePath + "-P" + partitionId;
-  }
-
-  /**
-   * Creates the path to read/write raw data buffers of a given partition
-   * from/to.
-   *
-   * @param basePath path prefix to create the actual path from
-   * @param partitionId id of the partition
-   * @return path to read/write raw data buffer to/from
-   */
-  private static String getBuffersPath(String basePath, int partitionId) {
-    return getPath(basePath, partitionId) + "_buffers";
-  }
-
-  /**
    * Writes a single raw entry to a given output stream.
    *
    * @param entry entry to write to output
@@ -361,26 +392,26 @@
    * Loads data of a partition into data store. Returns number of bytes loaded.
    *
    * @param partitionId id of the partition to load its data
-   * @param path path from which data should be loaded
+   * @param ioThreadId id of the IO thread performing the load
+   * @param index data index chain for the data to load
    * @return number of bytes loaded from disk to memory
    * @throws IOException
    */
-  protected abstract long loadInMemoryPartitionData(int partitionId,
-                                                    String path)
-      throws IOException;
+  protected abstract long loadInMemoryPartitionData(
+      int partitionId, int ioThreadId, DataIndex index) throws IOException;
 
   /**
    * Offloads data of a partition in data store to disk. Returns the number of
    * bytes offloaded to disk
    *
    * @param partitionId id of the partition to offload to disk
-   * @param path path to which data should be offloaded
+   * @param ioThreadId id of the IO thread performing the offload
+   * @param index data index chain for the data to offload
    * @return number of bytes offloaded from memory to disk
    * @throws IOException
    */
-  protected abstract long offloadInMemoryPartitionData(int partitionId,
-                                                       String path)
-      throws IOException;
+  protected abstract long offloadInMemoryPartitionData(
+      int partitionId, int ioThreadId, DataIndex index) throws IOException;
 
   /**
    * Gets the size of a given entry in bytes.
@@ -396,6 +427,6 @@
    * @param partitionId id of the partition to add the data to
    * @param entry input entry to add to the data store
    */
-  protected abstract void addEntryToImMemoryPartitionData(int partitionId,
+  protected abstract void addEntryToInMemoryPartitionData(int partitionId,
                                                           T entry);
 }
diff --git a/giraph-core/src/main/java/org/apache/giraph/ooc/data/DiskBackedEdgeStore.java b/giraph-core/src/main/java/org/apache/giraph/ooc/data/DiskBackedEdgeStore.java
index 53de52f..e727fbd 100644
--- a/giraph-core/src/main/java/org/apache/giraph/ooc/data/DiskBackedEdgeStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/ooc/data/DiskBackedEdgeStore.java
@@ -21,25 +21,18 @@
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.edge.EdgeStore;
 import org.apache.giraph.ooc.OutOfCoreEngine;
+import org.apache.giraph.ooc.persistence.DataIndex;
+import org.apache.giraph.ooc.persistence.OutOfCoreDataAccessor;
 import org.apache.giraph.utils.ByteArrayVertexIdEdges;
 import org.apache.giraph.utils.VertexIdEdges;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.log4j.Logger;
 
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
 import java.io.DataInput;
-import java.io.DataInputStream;
 import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
 import java.io.IOException;
 
-import static com.google.common.base.Preconditions.checkState;
-
 /**
  * Implementation of an edge-store used for out-of-core mechanism.
  *
@@ -49,7 +42,7 @@
  */
 public class DiskBackedEdgeStore<I extends WritableComparable,
     V extends Writable, E extends Writable>
-    extends OutOfCoreDataManager<VertexIdEdges<I, E>>
+    extends DiskBackedDataStore<VertexIdEdges<I, E>>
     implements EdgeStore<I, V, E> {
   /** Class logger. */
   private static final Logger LOG = Logger.getLogger(DiskBackedEdgeStore.class);
@@ -57,8 +50,6 @@
   private final EdgeStore<I, V, E> edgeStore;
   /** Configuration */
   private final ImmutableClassesGiraphConfiguration<I, V, E> conf;
-  /** Out-of-core engine */
-  private final OutOfCoreEngine oocEngine;
 
   /**
    * Constructor
@@ -72,10 +63,9 @@
       EdgeStore<I, V, E> edgeStore,
       ImmutableClassesGiraphConfiguration<I, V, E> conf,
       OutOfCoreEngine oocEngine) {
-    super(conf);
+    super(conf, oocEngine);
     this.edgeStore = edgeStore;
     this.conf = conf;
-    this.oocEngine = oocEngine;
   }
 
   @Override
@@ -114,32 +104,25 @@
         "should not be called for DiskBackedEdgeStore!");
   }
 
-  /**
-   * Gets the path that should be used specifically for edge data.
-   *
-   * @param basePath path prefix to build the actual path from
-   * @return path to files specific for edge data
-   */
-  private static String getPath(String basePath) {
-    return basePath + "_edge_store";
+  @Override
+  public long loadPartitionData(int partitionId)
+      throws IOException {
+    return loadPartitionDataProxy(partitionId,
+        new DataIndex().addIndex(DataIndex.TypeIndexEntry.EDGE_STORE));
   }
 
   @Override
-  public long loadPartitionData(int partitionId, String basePath)
+  public long offloadPartitionData(int partitionId)
       throws IOException {
-    return super.loadPartitionData(partitionId, getPath(basePath));
+    return offloadPartitionDataProxy(partitionId,
+        new DataIndex().addIndex(DataIndex.TypeIndexEntry.EDGE_STORE));
   }
 
   @Override
-  public long offloadPartitionData(int partitionId, String basePath)
+  public long offloadBuffers(int partitionId)
       throws IOException {
-    return super.offloadPartitionData(partitionId, getPath(basePath));
-  }
-
-  @Override
-  public long offloadBuffers(int partitionId, String basePath)
-      throws IOException {
-    return super.offloadBuffers(partitionId, getPath(basePath));
+    return offloadBuffersProxy(partitionId,
+        new DataIndex().addIndex(DataIndex.TypeIndexEntry.EDGE_STORE));
   }
 
   @Override
@@ -157,44 +140,31 @@
   }
 
   @Override
-  protected long loadInMemoryPartitionData(int partitionId, String path)
-      throws IOException {
+  protected long loadInMemoryPartitionData(
+      int partitionId, int ioThreadId, DataIndex index) throws IOException {
     long numBytes = 0;
-    File file = new File(path);
-    if (file.exists()) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("loadInMemoryPartitionData: loading edge data for " +
-            "partition " + partitionId + " from " + file.getAbsolutePath());
-      }
-      FileInputStream fis = new FileInputStream(file);
-      BufferedInputStream bis = new BufferedInputStream(fis);
-      DataInputStream dis = new DataInputStream(bis);
-      edgeStore.readPartitionEdgeStore(partitionId, dis);
-      dis.close();
-      numBytes = file.length();
-      checkState(file.delete(), "loadInMemoryPartitionData: failed to delete " +
-          "%s.", file.getAbsoluteFile());
+    if (hasPartitionDataOnFile.remove(partitionId)) {
+      OutOfCoreDataAccessor.DataInputWrapper inputWrapper =
+          oocEngine.getDataAccessor().prepareInput(ioThreadId, index.copy());
+      edgeStore.readPartitionEdgeStore(partitionId,
+          inputWrapper.getDataInput());
+      numBytes = inputWrapper.finalizeInput(true);
     }
     return numBytes;
   }
 
   @Override
-  protected long offloadInMemoryPartitionData(int partitionId, String path)
-      throws IOException {
+  protected long offloadInMemoryPartitionData(
+      int partitionId, int ioThreadId, DataIndex index) throws IOException {
     long numBytes = 0;
     if (edgeStore.hasEdgesForPartition(partitionId)) {
-      File file = new File(path);
-      checkState(!file.exists(), "offloadInMemoryPartitionData: edge store " +
-          "file %s already exist", file.getAbsoluteFile());
-      checkState(file.createNewFile(),
-          "offloadInMemoryPartitionData: cannot create edge store file %s",
-          file.getAbsoluteFile());
-      FileOutputStream fos = new FileOutputStream(file);
-      BufferedOutputStream bos = new BufferedOutputStream(fos);
-      DataOutputStream dos = new DataOutputStream(bos);
-      edgeStore.writePartitionEdgeStore(partitionId, dos);
-      dos.close();
-      numBytes = dos.size();
+      OutOfCoreDataAccessor.DataOutputWrapper outputWrapper =
+          oocEngine.getDataAccessor().prepareOutput(ioThreadId, index.copy(),
+              false);
+      edgeStore.writePartitionEdgeStore(partitionId,
+          outputWrapper.getDataOutput());
+      numBytes = outputWrapper.finalizeOutput();
+      hasPartitionDataOnFile.add(partitionId);
     }
     return numBytes;
   }
@@ -205,7 +175,7 @@
   }
 
   @Override
-  protected void addEntryToImMemoryPartitionData(int partitionId,
+  protected void addEntryToInMemoryPartitionData(int partitionId,
                                                  VertexIdEdges<I, E> edges) {
     oocEngine.getMetaPartitionManager().addPartition(partitionId);
     edgeStore.addPartitionEdges(partitionId, edges);
diff --git a/giraph-core/src/main/java/org/apache/giraph/ooc/data/DiskBackedMessageStore.java b/giraph-core/src/main/java/org/apache/giraph/ooc/data/DiskBackedMessageStore.java
index 94ba83a..c8d0f79 100644
--- a/giraph-core/src/main/java/org/apache/giraph/ooc/data/DiskBackedMessageStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/ooc/data/DiskBackedMessageStore.java
@@ -21,6 +21,10 @@
 import org.apache.giraph.comm.messages.MessageStore;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.factories.MessageValueFactory;
+import org.apache.giraph.ooc.OutOfCoreEngine;
+import org.apache.giraph.ooc.persistence.DataIndex;
+import org.apache.giraph.ooc.persistence.DataIndex.NumericIndexEntry;
+import org.apache.giraph.ooc.persistence.OutOfCoreDataAccessor;
 import org.apache.giraph.utils.ByteArrayOneMessageToManyIds;
 import org.apache.giraph.utils.ByteArrayVertexIdMessages;
 import org.apache.giraph.utils.VertexIdMessages;
@@ -28,19 +32,10 @@
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.log4j.Logger;
 
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
 import java.io.DataInput;
-import java.io.DataInputStream;
 import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
 import java.io.IOException;
 
-import static com.google.common.base.Preconditions.checkState;
-
 /**
  * Implementation of a message store used for out-of-core mechanism.
  *
@@ -48,7 +43,7 @@
  * @param <M> Message data
  */
 public class DiskBackedMessageStore<I extends WritableComparable,
-    M extends Writable> extends OutOfCoreDataManager<VertexIdMessages<I, M>>
+    M extends Writable> extends DiskBackedDataStore<VertexIdMessages<I, M>>
     implements MessageStore<I, M> {
   /** Class logger. */
   private static final Logger LOG =
@@ -82,6 +77,7 @@
    * Constructor
    *
    * @param config Configuration
+   * @param oocEngine Out-of-core engine
    * @param messageStore In-memory message store for which out-of-core message
    *                     store would be wrapper
    * @param useMessageCombiner Whether message combiner is used for this message
@@ -90,9 +86,10 @@
    */
   public DiskBackedMessageStore(ImmutableClassesGiraphConfiguration<I, ?, ?>
                                     config,
+                                OutOfCoreEngine oocEngine,
                                 MessageStore<I, M> messageStore,
                                 boolean useMessageCombiner, long superstep) {
-    super(config);
+    super(config, oocEngine);
     this.config = config;
     this.messageStore = messageStore;
     this.useMessageCombiner = useMessageCombiner;
@@ -140,43 +137,38 @@
     }
   }
 
-  /**
-   * Gets the path that should be used specifically for message data.
-   *
-   * @param basePath path prefix to build the actual path from
-   * @param superstep superstep for which message data should be stored
-   * @return path to files specific for message data
-   */
-  private static String getPath(String basePath, long superstep) {
-    return basePath + "_messages-S" + superstep;
-  }
 
   @Override
-  public long loadPartitionData(int partitionId, String basePath)
+  public long loadPartitionData(int partitionId)
       throws IOException {
     if (!useMessageCombiner) {
-      return super.loadPartitionData(partitionId, getPath(basePath, superstep));
+      return loadPartitionDataProxy(partitionId,
+          new DataIndex().addIndex(DataIndex.TypeIndexEntry.MESSAGE)
+              .addIndex(NumericIndexEntry.createSuperstepEntry(superstep)));
     } else {
       return 0;
     }
   }
 
   @Override
-  public long offloadPartitionData(int partitionId, String basePath)
+  public long offloadPartitionData(int partitionId)
       throws IOException {
     if (!useMessageCombiner) {
-      return
-          super.offloadPartitionData(partitionId, getPath(basePath, superstep));
+      return offloadPartitionDataProxy(partitionId,
+          new DataIndex().addIndex(DataIndex.TypeIndexEntry.MESSAGE)
+              .addIndex(NumericIndexEntry.createSuperstepEntry(superstep)));
     } else {
       return 0;
     }
   }
 
   @Override
-  public long offloadBuffers(int partitionId, String basePath)
+  public long offloadBuffers(int partitionId)
       throws IOException {
     if (!useMessageCombiner) {
-      return super.offloadBuffers(partitionId, getPath(basePath, superstep));
+      return offloadBuffersProxy(partitionId,
+          new DataIndex().addIndex(DataIndex.TypeIndexEntry.MESSAGE)
+              .addIndex(NumericIndexEntry.createSuperstepEntry(superstep)));
     } else {
       return 0;
     }
@@ -250,45 +242,31 @@
   }
 
   @Override
-  protected long loadInMemoryPartitionData(int partitionId, String basePath)
-      throws IOException {
+  protected long loadInMemoryPartitionData(int partitionId, int ioThreadId,
+                                           DataIndex index) throws IOException {
     long numBytes = 0;
-    File file = new File(basePath);
-    if (file.exists()) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("loadInMemoryPartitionData: loading message data for " +
-            "partition " + partitionId + " from " + file.getAbsolutePath());
-      }
-      FileInputStream fis = new FileInputStream(file);
-      BufferedInputStream bis = new BufferedInputStream(fis);
-      DataInputStream dis = new DataInputStream(bis);
-      messageStore.readFieldsForPartition(dis, partitionId);
-      dis.close();
-      numBytes = file.length();
-      checkState(file.delete(), "loadInMemoryPartitionData: failed to delete " +
-          "%s.", file.getAbsoluteFile());
+    if (hasPartitionDataOnFile.remove(partitionId)) {
+      OutOfCoreDataAccessor.DataInputWrapper inputWrapper =
+          oocEngine.getDataAccessor().prepareInput(ioThreadId, index.copy());
+      messageStore.readFieldsForPartition(inputWrapper.getDataInput(),
+          partitionId);
+      numBytes = inputWrapper.finalizeInput(true);
     }
     return numBytes;
   }
 
   @Override
-  protected long offloadInMemoryPartitionData(int partitionId, String basePath)
-      throws IOException {
+  protected long offloadInMemoryPartitionData(
+      int partitionId, int ioThreadId, DataIndex index) throws IOException {
     long numBytes = 0;
     if (messageStore.hasMessagesForPartition(partitionId)) {
-      File file = new File(basePath);
-      checkState(!file.exists(), "offloadInMemoryPartitionData: message store" +
-          " file %s already exist", file.getAbsoluteFile());
-      checkState(file.createNewFile(),
-          "offloadInMemoryPartitionData: cannot create message store file %s",
-          file.getAbsoluteFile());
-      FileOutputStream fileout = new FileOutputStream(file);
-      BufferedOutputStream bufferout = new BufferedOutputStream(fileout);
-      DataOutputStream outputStream = new DataOutputStream(bufferout);
-      messageStore.writePartition(outputStream, partitionId);
+      OutOfCoreDataAccessor.DataOutputWrapper outputWrapper =
+          oocEngine.getDataAccessor().prepareOutput(ioThreadId, index.copy(),
+              false);
+      messageStore.writePartition(outputWrapper.getDataOutput(), partitionId);
       messageStore.clearPartition(partitionId);
-      outputStream.close();
-      numBytes += outputStream.size();
+      numBytes = outputWrapper.finalizeOutput();
+      hasPartitionDataOnFile.add(partitionId);
     }
     return numBytes;
   }
@@ -299,7 +277,7 @@
   }
 
   @Override
-  protected void addEntryToImMemoryPartitionData(int partitionId,
+  protected void addEntryToInMemoryPartitionData(int partitionId,
                                                  VertexIdMessages<I, M>
                                                      messages) {
     messageStore.addPartitionMessages(partitionId, messages);
diff --git a/giraph-core/src/main/java/org/apache/giraph/ooc/data/DiskBackedPartitionStore.java b/giraph-core/src/main/java/org/apache/giraph/ooc/data/DiskBackedPartitionStore.java
index 2a5e47a..6b7822f 100644
--- a/giraph-core/src/main/java/org/apache/giraph/ooc/data/DiskBackedPartitionStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/ooc/data/DiskBackedPartitionStore.java
@@ -20,11 +20,12 @@
 
 import com.google.common.collect.Maps;
 import org.apache.giraph.bsp.BspService;
-import org.apache.giraph.bsp.CentralizedServiceWorker;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.edge.OutEdges;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.ooc.OutOfCoreEngine;
+import org.apache.giraph.ooc.persistence.DataIndex;
+import org.apache.giraph.ooc.persistence.OutOfCoreDataAccessor;
 import org.apache.giraph.partition.Partition;
 import org.apache.giraph.partition.PartitionStore;
 import org.apache.giraph.utils.ExtendedDataOutput;
@@ -35,25 +36,17 @@
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.log4j.Logger;
 
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
 import java.io.DataInput;
-import java.io.DataInputStream;
 import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.util.Map;
 
 import static com.google.common.base.Preconditions.checkNotNull;
-import static com.google.common.base.Preconditions.checkState;
 
 /**
  * Implementation of a partition-store used for out-of-core mechanism.
  * Partition store is responsible for partition data, as well as data buffers in
- * INPUT_SUPERSTEP ("raw data buffer" -- defined in OutOfCoreDataManager --
+ * INPUT_SUPERSTEP ("raw data buffer" -- defined in DiskBackedDataStore --
  * refers to vertex buffers in INPUT_SUPERSTEP).
  *
  * @param <I> Vertex id
@@ -62,7 +55,7 @@
  */
 public class DiskBackedPartitionStore<I extends WritableComparable,
     V extends Writable, E extends Writable>
-    extends OutOfCoreDataManager<ExtendedDataOutput>
+    extends DiskBackedDataStore<ExtendedDataOutput>
     implements PartitionStore<I, V, E> {
   /** Class logger. */
   private static final Logger LOG =
@@ -71,10 +64,6 @@
   private final ImmutableClassesGiraphConfiguration<I, V, E> conf;
   /** Job context (for progress) */
   private final Mapper<?, ?, ?, ?>.Context context;
-  /** Service worker */
-  private final CentralizedServiceWorker<I, V, E> serviceWorker;
-  /** Out-of-core engine */
-  private final OutOfCoreEngine oocEngine;
   /** In-memory partition store */
   private final PartitionStore<I, V, E> partitionStore;
   /**
@@ -99,21 +88,17 @@
    *                       partition store would be a wrapper
    * @param conf Configuration
    * @param context Job context
-   * @param serviceWorker Service worker
    * @param oocEngine Out-of-core engine
    */
   public DiskBackedPartitionStore(
       PartitionStore<I, V, E> partitionStore,
       ImmutableClassesGiraphConfiguration<I, V, E> conf,
       Mapper<?, ?, ?, ?>.Context context,
-      CentralizedServiceWorker<I, V, E> serviceWorker,
       OutOfCoreEngine oocEngine) {
-    super(conf);
+    super(conf, oocEngine);
     this.partitionStore = partitionStore;
     this.conf = conf;
     this.context = context;
-    this.serviceWorker = serviceWorker;
-    this.oocEngine = oocEngine;
   }
 
   @Override
@@ -222,36 +207,6 @@
   }
 
   /**
-   * Gets the path that should be used specifically for partition data.
-   *
-   * @param basePath path prefix to build the actual path from
-   * @return path to files specific for partition data
-   */
-  private static String getPath(String basePath) {
-    return basePath + "_partition";
-  }
-
-  /**
-   * Get the path to the file where vertices are stored.
-   *
-   * @param basePath path prefix to build the actual path from
-   * @return The path to the vertices file
-   */
-  private static String getVerticesPath(String basePath) {
-    return basePath + "_vertices";
-  }
-
-  /**
-   * Get the path to the file where edges are stored.
-   *
-   * @param basePath path prefix to build the actual path from
-   * @return The path to the edges file
-   */
-  private static String getEdgesPath(String basePath) {
-    return basePath + "_edges";
-  }
-
-  /**
    * Read vertex data from an input and initialize the vertex.
    *
    * @param in     The input stream
@@ -295,54 +250,42 @@
   }
 
   @Override
-  protected long loadInMemoryPartitionData(int partitionId, String path)
-      throws IOException {
+  protected long loadInMemoryPartitionData(int partitionId, int ioThreadId,
+                                           DataIndex index) throws IOException {
     long numBytes = 0;
     // Load vertices
-    File file = new File(getVerticesPath(path));
-    if (file.exists()) {
+    if (hasPartitionDataOnFile.remove(partitionId)) {
       Partition<I, V, E> partition = conf.createPartition(partitionId, context);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("loadInMemoryPartitionData: loading partition vertices " +
-            partitionId + " from " + file.getAbsolutePath());
-      }
-
-      FileInputStream fis = new FileInputStream(file);
-      BufferedInputStream bis = new BufferedInputStream(fis);
-      DataInputStream inputStream = new DataInputStream(bis);
-      long numVertices = inputStream.readLong();
+      OutOfCoreDataAccessor dataAccessor = oocEngine.getDataAccessor();
+      index.addIndex(DataIndex.TypeIndexEntry.PARTITION_VERTICES);
+      OutOfCoreDataAccessor.DataInputWrapper inputWrapper =
+          dataAccessor.prepareInput(ioThreadId, index.copy());
+      DataInput dataInput = inputWrapper.getDataInput();
+      long numVertices = dataInput.readLong();
       for (long i = 0; i < numVertices; ++i) {
         Vertex<I, V, E> vertex = conf.createVertex();
-        readVertexData(inputStream, vertex);
+        readVertexData(dataInput, vertex);
         partition.putVertex(vertex);
       }
-      inputStream.close();
-      numBytes += file.length();
-      checkState(file.delete(), "loadInMemoryPartitionData: failed to delete " +
-          "%s", file.getAbsolutePath());
+      numBytes += inputWrapper.finalizeInput(true);
 
       // Load edges
-      file = new File(getEdgesPath(path));
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("loadInMemoryPartitionData: loading partition edges " +
-            partitionId + " from " + file.getAbsolutePath());
-      }
-
-      fis = new FileInputStream(file);
-      bis = new BufferedInputStream(fis);
-      inputStream = new DataInputStream(bis);
+      index.removeLastIndex()
+          .addIndex(DataIndex.TypeIndexEntry.PARTITION_EDGES);
+      inputWrapper = dataAccessor.prepareInput(ioThreadId, index.copy());
+      dataInput = inputWrapper.getDataInput();
       for (int i = 0; i < numVertices; ++i) {
-        readOutEdges(inputStream, partition);
+        readOutEdges(dataInput, partition);
       }
-      inputStream.close();
-      numBytes += file.length();
       // If the graph is static and it is not INPUT_SUPERSTEP, keep the file
       // around.
+      boolean shouldDeleteEdges = false;
       if (!conf.isStaticGraph() ||
           oocEngine.getSuperstep() == BspService.INPUT_SUPERSTEP) {
-        checkState(file.delete(), "loadPartition: failed to delete %s",
-            file.getAbsolutePath());
+        shouldDeleteEdges = true;
       }
+      numBytes += inputWrapper.finalizeInput(shouldDeleteEdges);
+      index.removeLastIndex();
       partitionStore.addPartition(partition);
     }
     return numBytes;
@@ -354,7 +297,7 @@
   }
 
   @Override
-  protected void addEntryToImMemoryPartitionData(int partitionId,
+  protected void addEntryToInMemoryPartitionData(int partitionId,
                                                  ExtendedDataOutput vertices) {
     if (!partitionStore.hasPartition(partitionId)) {
       oocEngine.getMetaPartitionManager().addPartition(partitionId);
@@ -363,15 +306,17 @@
   }
 
   @Override
-  public long loadPartitionData(int partitionId, String basePath)
+  public long loadPartitionData(int partitionId)
       throws IOException {
-    return super.loadPartitionData(partitionId, getPath(basePath));
+    return loadPartitionDataProxy(partitionId,
+        new DataIndex().addIndex(DataIndex.TypeIndexEntry.PARTITION));
   }
 
   @Override
-  public long offloadPartitionData(int partitionId, String basePath)
+  public long offloadPartitionData(int partitionId)
       throws IOException {
-    return super.offloadPartitionData(partitionId, getPath(basePath));
+    return offloadPartitionDataProxy(partitionId,
+        new DataIndex().addIndex(DataIndex.TypeIndexEntry.PARTITION));
   }
 
   /**
@@ -409,61 +354,44 @@
   }
 
   @Override
-  protected long offloadInMemoryPartitionData(int partitionId, String path)
-      throws IOException {
+  protected long offloadInMemoryPartitionData(
+      int partitionId, int ioThreadId, DataIndex index) throws IOException {
     long numBytes = 0;
     if (partitionStore.hasPartition(partitionId)) {
+      OutOfCoreDataAccessor dataAccessor = oocEngine.getDataAccessor();
       partitionVertexCount.put(partitionId,
           partitionStore.getPartitionVertexCount(partitionId));
       partitionEdgeCount.put(partitionId,
           partitionStore.getPartitionEdgeCount(partitionId));
       Partition<I, V, E> partition =
           partitionStore.removePartition(partitionId);
-      File file = new File(getVerticesPath(path));
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("offloadInMemoryPartitionData: writing partition vertices " +
-            partitionId + " to " + file.getAbsolutePath());
-      }
-      checkState(!file.exists(), "offloadInMemoryPartitionData: partition " +
-          "store file %s already exist", file.getAbsoluteFile());
-      checkState(file.createNewFile(),
-          "offloadInMemoryPartitionData: file %s already exists.",
-          file.getAbsolutePath());
-
-      FileOutputStream fileout = new FileOutputStream(file);
-      BufferedOutputStream bufferout = new BufferedOutputStream(fileout);
-      DataOutputStream outputStream = new DataOutputStream(bufferout);
-      outputStream.writeLong(partition.getVertexCount());
+      index.addIndex(DataIndex.TypeIndexEntry.PARTITION_VERTICES);
+      OutOfCoreDataAccessor.DataOutputWrapper outputWrapper =
+          dataAccessor.prepareOutput(ioThreadId, index.copy(), false);
+      DataOutput dataOutput = outputWrapper.getDataOutput();
+      dataOutput.writeLong(partition.getVertexCount());
       for (Vertex<I, V, E> vertex : partition) {
-        writeVertexData(outputStream, vertex);
+        writeVertexData(dataOutput, vertex);
       }
-      outputStream.close();
-      numBytes += outputStream.size();
-
+      numBytes += outputWrapper.finalizeOutput();
+      index.removeLastIndex();
       // Avoid writing back edges if we have already written them once and
       // the graph is not changing.
       // If we are in the input superstep, we need to write the files
       // at least the first time, even though the graph is static.
-      file = new File(getEdgesPath(path));
+      index.addIndex(DataIndex.TypeIndexEntry.PARTITION_EDGES);
       if (oocEngine.getSuperstep() == BspServiceWorker.INPUT_SUPERSTEP ||
-          partitionVertexCount.get(partitionId) == null ||
-          partitionVertexCount.get(partitionId) != partition.getVertexCount() ||
-          !conf.isStaticGraph() || !file.exists()) {
-        checkState(file.createNewFile(), "offloadInMemoryPartitionData: file " +
-            "%s already exists.", file.getAbsolutePath());
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("offloadInMemoryPartitionData: writing partition edges " +
-              partitionId + " to " + file.getAbsolutePath());
-        }
-        fileout = new FileOutputStream(file);
-        bufferout = new BufferedOutputStream(fileout);
-        outputStream = new DataOutputStream(bufferout);
+          !conf.isStaticGraph() ||
+          !dataAccessor.dataExist(ioThreadId, index)) {
+        outputWrapper = dataAccessor.prepareOutput(ioThreadId, index.copy(),
+            false);
         for (Vertex<I, V, E> vertex : partition) {
-          writeOutEdges(outputStream, vertex);
+          writeOutEdges(outputWrapper.getDataOutput(), vertex);
         }
-        outputStream.close();
-        numBytes += outputStream.size();
+        numBytes += outputWrapper.finalizeOutput();
       }
+      index.removeLastIndex();
+      hasPartitionDataOnFile.add(partitionId);
     }
     return numBytes;
   }
@@ -475,9 +403,10 @@
   }
 
   @Override
-  public long offloadBuffers(int partitionId, String basePath)
+  public long offloadBuffers(int partitionId)
       throws IOException {
-    return super.offloadBuffers(partitionId, getPath(basePath));
+    return offloadBuffersProxy(partitionId,
+        new DataIndex().addIndex(DataIndex.TypeIndexEntry.PARTITION));
   }
 
   @Override
diff --git a/giraph-core/src/main/java/org/apache/giraph/ooc/data/MetaPartitionManager.java b/giraph-core/src/main/java/org/apache/giraph/ooc/data/MetaPartitionManager.java
index 1332a3a..64e3aed 100644
--- a/giraph-core/src/main/java/org/apache/giraph/ooc/data/MetaPartitionManager.java
+++ b/giraph-core/src/main/java/org/apache/giraph/ooc/data/MetaPartitionManager.java
@@ -99,6 +99,21 @@
    */
   private final AtomicDouble lowestGraphFractionInMemory =
       new AtomicDouble(1);
+  /**
+   * Map of partition ids to their indices. index of a partition is the order
+   * with which the partition has been inserted. Partitions are indexed as 0, 1,
+   * 2, etc. This indexing is later used to find the id of the IO thread who is
+   * responsible for handling a partition. Partitions are assigned to IO threads
+   * in a round-robin fashion based on their indices.
+   */
+  private final ConcurrentMap<Integer, Integer> partitionIndex =
+      Maps.newConcurrentMap();
+  /**
+   * Sequential counter used to assign indices to partitions as they are added
+   */
+  private final AtomicInteger indexCounter = new AtomicInteger(0);
+  /** How many disks (i.e. IO threads) do we have? */
+  private final int numIOThreads;
 
   /**
    * Constructor
@@ -117,6 +132,7 @@
     }
     this.oocEngine = oocEngine;
     this.randomGenerator = new Random();
+    this.numIOThreads = numIOThreads;
   }
 
   /**
@@ -131,7 +147,7 @@
   /**
    * Get total number of partitions
    *
-   * @return total number of partition
+   * @return total number of partitions
    */
   public int getNumPartitions() {
     return partitions.size();
@@ -175,6 +191,18 @@
   }
 
   /**
+   * Get the thread id that is responsible for a particular partition
+   *
+   * @param partitionId id of the given partition
+   * @return id of the thread responsible for the given partition
+   */
+  public int getOwnerThreadId(int partitionId) {
+    Integer index = partitionIndex.get(partitionId);
+    checkState(index != null);
+    return index % numIOThreads;
+  }
+
+  /**
    * Add a partition
    *
    * @param partitionId id of a partition to add
@@ -184,8 +212,9 @@
     MetaPartition temp = partitions.putIfAbsent(partitionId, meta);
     // Check if the given partition is new
     if (temp == null) {
-      int ownerThread = oocEngine.getIOScheduler()
-          .getOwnerThreadId(partitionId);
+      int index = indexCounter.getAndIncrement();
+      checkState(partitionIndex.putIfAbsent(partitionId, index) == null);
+      int ownerThread = getOwnerThreadId(partitionId);
       perThreadPartitionDictionary.get(ownerThread).addPartition(meta);
       numInMemoryPartitions.getAndIncrement();
     }
@@ -199,7 +228,7 @@
    */
   public void removePartition(Integer partitionId) {
     MetaPartition meta = partitions.remove(partitionId);
-    int ownerThread = oocEngine.getIOScheduler().getOwnerThreadId(partitionId);
+    int ownerThread = getOwnerThreadId(partitionId);
     perThreadPartitionDictionary.get(ownerThread).removePartition(meta);
     checkState(!meta.isOnDisk());
     numInMemoryPartitions.getAndDecrement();
@@ -424,7 +453,7 @@
    */
   public void markPartitionAsInProcess(int partitionId) {
     MetaPartition meta = partitions.get(partitionId);
-    int ownerThread = oocEngine.getIOScheduler().getOwnerThreadId(partitionId);
+    int ownerThread = getOwnerThreadId(partitionId);
     synchronized (meta) {
       perThreadPartitionDictionary.get(ownerThread).removePartition(meta);
       meta.setProcessingState(ProcessingState.IN_PROCESS);
@@ -468,7 +497,7 @@
    */
   public void setPartitionIsProcessed(int partitionId) {
     MetaPartition meta = partitions.get(partitionId);
-    int ownerThread = oocEngine.getIOScheduler().getOwnerThreadId(partitionId);
+    int ownerThread = getOwnerThreadId(partitionId);
     synchronized (meta) {
       perThreadPartitionDictionary.get(ownerThread).removePartition(meta);
       meta.setProcessingState(ProcessingState.PROCESSED);
@@ -508,7 +537,7 @@
   public void doneLoadingPartition(int partitionId, long superstep) {
     MetaPartition meta = partitions.get(partitionId);
     numInMemoryPartitions.getAndIncrement();
-    int owner = oocEngine.getIOScheduler().getOwnerThreadId(partitionId);
+    int owner = getOwnerThreadId(partitionId);
     synchronized (meta) {
       perThreadPartitionDictionary.get(owner).removePartition(meta);
       meta.setPartitionState(StorageState.IN_MEM);
@@ -535,8 +564,7 @@
    */
   public boolean startOffloadingMessages(int partitionId) {
     MetaPartition meta = partitions.get(partitionId);
-    int ownerThread =
-        oocEngine.getIOScheduler().getOwnerThreadId(partitionId);
+    int ownerThread = getOwnerThreadId(partitionId);
     synchronized (meta) {
       if (meta.getIncomingMessagesState() == StorageState.IN_MEM) {
         perThreadPartitionDictionary.get(ownerThread).removePartition(meta);
@@ -558,8 +586,7 @@
    */
   public void doneOffloadingMessages(int partitionId) {
     MetaPartition meta = partitions.get(partitionId);
-    int ownerThread =
-        oocEngine.getIOScheduler().getOwnerThreadId(partitionId);
+    int ownerThread = getOwnerThreadId(partitionId);
     synchronized (meta) {
       perThreadPartitionDictionary.get(ownerThread).removePartition(meta);
       meta.setIncomingMessagesState(StorageState.ON_DISK);
@@ -598,7 +625,7 @@
    */
   public boolean startOffloadingPartition(int partitionId) {
     MetaPartition meta = partitions.get(partitionId);
-    int owner = oocEngine.getIOScheduler().getOwnerThreadId(partitionId);
+    int owner = getOwnerThreadId(partitionId);
     synchronized (meta) {
       if (meta.getProcessingState() != ProcessingState.IN_PROCESS &&
           (meta.getPartitionState() == StorageState.IN_MEM ||
@@ -624,7 +651,7 @@
     numInMemoryPartitions.getAndDecrement();
     updateGraphFractionInMemory();
     MetaPartition meta = partitions.get(partitionId);
-    int owner = oocEngine.getIOScheduler().getOwnerThreadId(partitionId);
+    int owner = getOwnerThreadId(partitionId);
     synchronized (meta) {
       perThreadPartitionDictionary.get(owner).removePartition(meta);
       meta.setPartitionState(StorageState.ON_DISK);
@@ -639,8 +666,7 @@
    */
   public void resetPartitions() {
     for (MetaPartition meta : partitions.values()) {
-      int owner =
-          oocEngine.getIOScheduler().getOwnerThreadId(meta.getPartitionId());
+      int owner = getOwnerThreadId(meta.getPartitionId());
       perThreadPartitionDictionary.get(owner).removePartition(meta);
       meta.resetPartition();
       perThreadPartitionDictionary.get(owner).addPartition(meta);
@@ -659,8 +685,7 @@
    */
   public void resetMessages() {
     for (MetaPartition meta : partitions.values()) {
-      int owner =
-          oocEngine.getIOScheduler().getOwnerThreadId(meta.getPartitionId());
+      int owner = getOwnerThreadId(meta.getPartitionId());
       perThreadPartitionDictionary.get(owner).removePartition(meta);
       meta.resetMessages();
       if (meta.getPartitionState() == StorageState.IN_MEM &&
diff --git a/giraph-core/src/main/java/org/apache/giraph/ooc/persistence/DataIndex.java b/giraph-core/src/main/java/org/apache/giraph/ooc/persistence/DataIndex.java
new file mode 100644
index 0000000..d44204b
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/ooc/persistence/DataIndex.java
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.ooc.persistence;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Index chain used in out-of-core data accessor object (DAO) to access
+ * serialized data.
+ */
+public class DataIndex {
+  /** Chain of data indices */
+  private final List<DataIndexEntry> indexList = new ArrayList<>(5);
+
+  /**
+   * Add an index to the index chain
+   *
+   * @param entry the entry to add to the chain
+   * @return the index chain itself
+   */
+  public DataIndex addIndex(DataIndexEntry entry) {
+    indexList.add(entry);
+    return this;
+  }
+
+  /**
+   * Remove/Pop the last index in the index chain
+   *
+   * @return the index chain itself
+   */
+  public DataIndex removeLastIndex() {
+    indexList.remove(indexList.size() - 1);
+    return this;
+  }
+
+  /**
+   * Create a copy of the existing DataIndex
+   *
+   * @return a copy of the existing index chain
+   */
+  public DataIndex copy() {
+    DataIndex index = new DataIndex();
+    for (DataIndexEntry entry : indexList) {
+      index.indexList.add(entry);
+    }
+    return index;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (!(obj instanceof DataIndex)) {
+      return false;
+    }
+    DataIndex dataIndex = (DataIndex) obj;
+    return indexList.equals(dataIndex.indexList);
+  }
+
+  @Override
+  public int hashCode() {
+    return indexList.hashCode();
+  }
+
+  @Override
+  public String toString() {
+    StringBuffer sb = new StringBuffer();
+    for (DataIndexEntry entry : indexList) {
+      sb.append(entry);
+    }
+    return sb.toString();
+  }
+
+  /** Interface to unify different types of entries used as index chain */
+  public interface DataIndexEntry { }
+
+  /**
+   * Different static types of index chain entry
+   */
+  public enum TypeIndexEntry implements DataIndexEntry {
+    /** The whole partition */
+    PARTITION("_partition"),
+    /** Partition vertices */
+    PARTITION_VERTICES("_vertices"),
+    /** Partition edges */
+    PARTITION_EDGES("_edges"),
+    /** Partition messages */
+    MESSAGE("_messages"),
+    /** Edges stored in edge store for a partition */
+    EDGE_STORE("_edge_store"),
+    /** Raw data buffer (refer to DiskBackedDataStore) */
+    BUFFER("_buffer");
+
+    /** String realization of entry type */
+    private final String name;
+
+    /**
+     * Constructor
+     *
+     * @param name name of the type
+     */
+    TypeIndexEntry(String name) {
+      this.name = name;
+    }
+
+    public String getName() {
+      return name;
+    }
+
+    @Override
+    public String toString() {
+      return name;
+    }
+  }
+
+  /**
+   * Class representing any index chain that depends on something with id.
+   * Generally this is used for identifying indices in two types:
+   *  - Index entry based on superstep id ('S' and the superstep number)
+   *  - Index entry based on partition id ('P' and the partition id)
+   */
+  public static final class NumericIndexEntry implements DataIndexEntry {
+    /** Type of index */
+    private final char type;
+    /** Id of the index associated with the specified type */
+    private final long id;
+
+    /**
+     * Constructor
+     *
+     * @param type type of index (for now 'S' for superstep, or 'P' for
+     *             partition)
+     * @param id id of the index associated with the given type
+     */
+    private NumericIndexEntry(char type, long id) {
+      this.type = type;
+      this.id = id;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (!(obj instanceof NumericIndexEntry)) {
+        return false;
+      }
+      NumericIndexEntry index = (NumericIndexEntry) obj;
+      return index.type == type && index.id == id;
+    }
+
+    @Override
+    public int hashCode() {
+      int result = 17;
+      result = result * 37 + type;
+      result = result * 37 + (int) id;
+      result = result * 37 + (int) (id >> 32);
+      return result;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("_%c%d", type, id);
+    }
+
+    /**
+     * Create a data index entry for a given partition
+     *
+     * @param partitionId id of the partition
+     * @return data index entry for a given partition
+     */
+    public static NumericIndexEntry createPartitionEntry(int partitionId) {
+      return new NumericIndexEntry('P', partitionId);
+    }
+
+    /**
+     * Create a data index entry for a given superstep
+     *
+     * @param superstep the superstep number
+     * @return data index entry for a given superstep
+     */
+    public static NumericIndexEntry createSuperstepEntry(long superstep) {
+      return new NumericIndexEntry('S', superstep);
+    }
+  }
+}
diff --git a/giraph-core/src/main/java/org/apache/giraph/ooc/persistence/LocalDiskDataAccessor.java b/giraph-core/src/main/java/org/apache/giraph/ooc/persistence/LocalDiskDataAccessor.java
new file mode 100644
index 0000000..2e42906
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/ooc/persistence/LocalDiskDataAccessor.java
@@ -0,0 +1,252 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.ooc.persistence;
+
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.KryoDataInput;
+import com.esotericsoftware.kryo.io.KryoDataOutput;
+import com.esotericsoftware.kryo.io.Output;
+import com.esotericsoftware.kryo.io.UnsafeInput;
+import com.esotericsoftware.kryo.io.UnsafeOutput;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.IntConfOption;
+import org.apache.log4j.Logger;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+
+import static com.google.common.base.Preconditions.checkState;
+import static org.apache.giraph.conf.GiraphConstants.ONE_MB;
+
+/**
+ * Data accessor object to read/write data in local disk.
+ * Note: This class assumes that the data are partitioned across IO threads,
+ *       i.e. each part of data can be accessed by one and only one IO thread
+ *       throughout the execution. Also, each IO thread reads a particular
+ *       type of data completely and, only then, it can read other type of data;
+ *       i.e. an IO thread cannot be used to read two different files at the
+ *       same time. These assumptions are based on the assumptions that the
+ *       current out-of-core mechanism is designed for.
+ */
+public class LocalDiskDataAccessor implements OutOfCoreDataAccessor {
+  /**
+   * Size of the buffer used for (de)serializing data when reading/writing
+   * from/to disk
+   */
+  public static final IntConfOption OOC_DISK_BUFFER_SIZE =
+      new IntConfOption("graph.oocDiskBufferSize", 4 * ONE_MB,
+          "size of the buffer when (de)serializing data for reading/writing " +
+              "from/to disk");
+
+  /** Class logger */
+  private static final Logger LOG =
+      Logger.getLogger(LocalDiskDataAccessor.class);
+  /**
+   * In-memory buffer used for (de)serializing data when reading/writing
+   * from/to disk using Kryo
+   */
+  private final byte[][] perThreadBuffers;
+  /** Path prefix for different disks */
+  private final String[] basePaths;
+  /** How many disks (i.e. IO threads) do we have? */
+  private final int numDisks;
+
+  /**
+   * Constructor
+   *
+   * @param conf Configuration
+   */
+  public LocalDiskDataAccessor(
+      ImmutableClassesGiraphConfiguration<?, ?, ?> conf) {
+    // Take advantage of multiple disks
+    String[] userPaths = GiraphConstants.PARTITIONS_DIRECTORY.getArray(conf);
+    this.numDisks = userPaths.length;
+    if (!GiraphConstants.NUM_OUT_OF_CORE_THREADS.isDefaultValue(conf) ||
+        GiraphConstants.NUM_OUT_OF_CORE_THREADS.get(conf) != numDisks) {
+      LOG.warn("LocalDiskDataAccessor: with this data accessor, number of " +
+          "out-of-core threads is only specified by the number of " +
+          "directories given by 'giraph.partitionsDirectory' flag! Now using " +
+          numDisks + " IO threads!");
+    }
+    this.basePaths = new String[numDisks];
+    int ptr = 0;
+    String jobId = conf.getJobId();
+    for (String path : userPaths) {
+      File file = new File(path);
+      if (!file.exists()) {
+        checkState(file.mkdirs(), "LocalDiskDataAccessor: cannot create " +
+            "directory " + file.getAbsolutePath());
+      }
+      basePaths[ptr] = path + "/" + jobId;
+      ptr++;
+    }
+    final int diskBufferSize = OOC_DISK_BUFFER_SIZE.get(conf);
+    this.perThreadBuffers = new byte[numDisks][diskBufferSize];
+  }
+
+  @Override
+  public void initialize() { }
+
+  @Override
+  public void shutdown() {
+    for (String path : basePaths) {
+      File file = new File(path).getParentFile();
+      for (String subFileName : file.list()) {
+        File subFile = new File(file.getPath(), subFileName);
+        checkState(subFile.delete(), "shutdown: cannot delete file %s",
+            subFile.getAbsoluteFile());
+      }
+      checkState(file.delete(), "shutdown: cannot delete directory %s",
+          file.getAbsoluteFile());
+    }
+  }
+
+  @Override
+  public int getNumAccessorThreads() {
+    return numDisks;
+  }
+
+  @Override
+  public DataInputWrapper prepareInput(int threadId, DataIndex index)
+      throws IOException {
+    return new LocalDiskDataInputWrapper(basePaths[threadId] + index.toString(),
+        perThreadBuffers[threadId]);
+  }
+
+  @Override
+  public DataOutputWrapper prepareOutput(
+      int threadId, DataIndex index, boolean shouldAppend) throws IOException {
+    return new LocalDiskDataOutputWrapper(
+        basePaths[threadId] + index.toString(), shouldAppend,
+        perThreadBuffers[threadId]);
+  }
+
+  @Override
+  public boolean dataExist(int threadId, DataIndex index) {
+    return new File(basePaths[threadId] + index.toString()).exists();
+  }
+
+  /** Implementation of <code>DataInput</code> wrapper for local disk reader */
+  private static class LocalDiskDataInputWrapper implements DataInputWrapper {
+    /** File used to read the data from */
+    private final File file;
+    /** Kryo's handle to read the data */
+    private final Input input;
+
+    /**
+     * Constructor
+     *
+     * @param fileName file name
+     * @param buffer reusable byte buffer that will be used in Kryo's Input
+     *               reader
+     * @throws IOException
+     */
+    @edu.umd.cs.findbugs.annotations.SuppressWarnings(
+        "OBL_UNSATISFIED_OBLIGATION")
+    LocalDiskDataInputWrapper(String fileName, byte[] buffer)
+        throws IOException {
+      file = new File(fileName);
+      LOG.info("LocalDiskDataInputWrapper: obtaining a data input from local " +
+          "file " + file.getAbsolutePath());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("LocalDiskDataInputWrapper: obtaining a data input from " +
+            "local file " + file.getAbsolutePath());
+      }
+      input = new UnsafeInput(buffer);
+      input.setInputStream(new FileInputStream(
+          new RandomAccessFile(file, "r").getFD()));
+    }
+
+    @Override
+    public DataInput getDataInput() {
+      return new KryoDataInput(input);
+    }
+
+    @Override
+    public long finalizeInput(boolean deleteOnClose) {
+      input.close();
+      long count = input.total();
+      checkState(!deleteOnClose || file.delete(),
+          "finalizeInput: failed to delete %s.", file.getAbsoluteFile());
+      return count;
+    }
+  }
+
+  /** Implementation of <code>DataOutput</code> wrapper for local disk writer */
+  private static class LocalDiskDataOutputWrapper implements DataOutputWrapper {
+    /** File used to write the data to */
+    private final File file;
+    /** Kryo's handle to write the date */
+    private final Output output;
+
+    /**
+     * Constructor
+     *
+     * @param fileName file name
+     * @param shouldAppend whether the <code>DataOutput</code> should be used
+     *                     for appending to already existing files
+     * @param buffer reusable byte buffer that will be used in Kryo's Output
+     *               writer
+     * @throws IOException
+     */
+    @edu.umd.cs.findbugs.annotations.SuppressWarnings(
+        "OBL_UNSATISFIED_OBLIGATION")
+    LocalDiskDataOutputWrapper(String fileName, boolean shouldAppend,
+                               byte[] buffer) throws IOException {
+      file = new File(fileName);
+      LOG.info("LocalDiskDataOutputWrapper: obtaining a data output from " +
+          "local file " + file.getAbsolutePath());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("LocalDiskDataOutputWrapper: obtaining a data output from " +
+            "local file " + file.getAbsolutePath());
+        if (!shouldAppend) {
+          checkState(!file.exists(), "LocalDiskDataOutputWrapper: file %s " +
+              "already exist", file.getAbsoluteFile());
+          checkState(file.createNewFile(), "LocalDiskDataOutputWrapper: " +
+              "cannot create file %s", file.getAbsolutePath());
+        }
+      }
+      output = new UnsafeOutput(buffer);
+      RandomAccessFile raf = new RandomAccessFile(file, "rw");
+      if (shouldAppend) {
+        raf.seek(file.length());
+      }
+      output.setOutputStream(new FileOutputStream(raf.getFD()));
+    }
+
+    @Override
+    public DataOutput getDataOutput() {
+      return new KryoDataOutput(output);
+    }
+
+
+    @Override
+    public long finalizeOutput() {
+      output.close();
+      long count = output.total();
+      return count;
+    }
+  }
+}
diff --git a/giraph-core/src/main/java/org/apache/giraph/ooc/persistence/OutOfCoreDataAccessor.java b/giraph-core/src/main/java/org/apache/giraph/ooc/persistence/OutOfCoreDataAccessor.java
new file mode 100644
index 0000000..d4ddc62
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/ooc/persistence/OutOfCoreDataAccessor.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.ooc.persistence;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ * Interface representing data accessor object (DAO) used as persistence layer
+ * in out-of-core mechanism.
+ * Note: any class implementing this interface should have one and only one
+ *       constructor taking one and only one argument of type
+ *       <code>ImmutableClassesGiraphConfiguration</code>
+ */
+public interface OutOfCoreDataAccessor {
+  /** Initialize the DAO */
+  void initialize();
+
+  /** Shut down the DAO */
+  void shutdown();
+
+  /**
+   * @return the number of threads involved in data persistence
+   */
+  int getNumAccessorThreads();
+
+  /**
+   * Prepare a wrapper containing <code>DataInput</code> representation for a
+   * given thread involved in persistence for a given index chain for data.
+   *
+   * @param threadId id of the thread involved in persistence
+   * @param index index chain of the data to access the serialized data form
+   * @return the wrapper for <code>DataInput</code> representation of data
+   * @throws IOException
+   */
+  DataInputWrapper prepareInput(int threadId, DataIndex index)
+      throws IOException;
+
+  /**
+   * Prepare a wrapper containing <code>DataOutput</code> representation for a
+   * given thread involved in persistence for a given index chain for data.
+   *
+   * @param threadId id of the thread involved in persistence
+   * @param index index chain of the data to access the serialized data form
+   * @param shouldAppend whether the <code>DataOutput</code> should be used for
+   *                     appending to already existing data for the given index
+   *                     or the <code>DataOutput</code> should create new
+   *                     instance to store serialized data
+   * @return the wrapper for <code>DataOutput</code> representation of data
+   * @throws IOException
+   */
+  DataOutputWrapper prepareOutput(int threadId, DataIndex index,
+                                  boolean shouldAppend) throws IOException;
+
+  /**
+   * Whether the data for the given thread and index chain exists?
+   *
+   * @param threadId id of the thread involved in persistence
+   * @param index index chain used to access the data
+   * @return True if the data exists for the given index chain for the given
+   *         thread, False otherwise
+   */
+  boolean dataExist(int threadId, DataIndex index);
+
+  /** Interface to wrap <code>DataInput</code> */
+  interface DataInputWrapper {
+    /**
+     * @return the <code>DataInput</code>
+     */
+    DataInput getDataInput();
+
+    /**
+     * Finalize and close the <code>DataInput</code> used for persistence.
+     *
+     * @param deleteOnClose whether the source of <code>DataInput</code>
+     *                      should be deleted on closing/finalizing
+     * @return number of bytes read from <code>DataInput</code> since it was
+     *         opened
+     */
+    long finalizeInput(boolean deleteOnClose);
+  }
+
+  /** Interface to warp <code>DataOutput</code> */
+  interface DataOutputWrapper {
+    /**
+     * @return the <code>DataOutput</code>
+     */
+    DataOutput getDataOutput();
+
+    /**
+     * Finalize and close the <code>DataOutput</code> used for persistence.
+     *
+     * @return number of bytes written to <code>DataOutput</code> since it was
+     *         opened
+     */
+    long finalizeOutput();
+  }
+}
diff --git a/giraph-core/src/main/java/org/apache/giraph/ooc/io/package-info.java b/giraph-core/src/main/java/org/apache/giraph/ooc/persistence/package-info.java
similarity index 84%
copy from giraph-core/src/main/java/org/apache/giraph/ooc/io/package-info.java
copy to giraph-core/src/main/java/org/apache/giraph/ooc/persistence/package-info.java
index 2230ec4..adf8dba 100644
--- a/giraph-core/src/main/java/org/apache/giraph/ooc/io/package-info.java
+++ b/giraph-core/src/main/java/org/apache/giraph/ooc/persistence/package-info.java
@@ -16,6 +16,7 @@
  * limitations under the License.
  */
 /**
- * Package of classes related to IO operations in out-of-core mechanism
+ * Package of classes related to IO abstraction or persistence layer used for
+ * out-of-core mechanism
  */
-package org.apache.giraph.ooc.io;
+package org.apache.giraph.ooc.persistence;
diff --git a/giraph-core/src/main/java/org/apache/giraph/ooc/FixedPartitionsOracle.java b/giraph-core/src/main/java/org/apache/giraph/ooc/policy/FixedPartitionsOracle.java
similarity index 95%
rename from giraph-core/src/main/java/org/apache/giraph/ooc/FixedPartitionsOracle.java
rename to giraph-core/src/main/java/org/apache/giraph/ooc/policy/FixedPartitionsOracle.java
index f7badcb..ffc5f7f 100644
--- a/giraph-core/src/main/java/org/apache/giraph/ooc/FixedPartitionsOracle.java
+++ b/giraph-core/src/main/java/org/apache/giraph/ooc/policy/FixedPartitionsOracle.java
@@ -16,14 +16,15 @@
  * limitations under the License.
  */
 
-package org.apache.giraph.ooc;
+package org.apache.giraph.ooc.policy;
 
 import com.sun.management.GarbageCollectionNotificationInfo;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.ooc.io.IOCommand;
-import org.apache.giraph.ooc.io.LoadPartitionIOCommand;
-import org.apache.giraph.ooc.io.StorePartitionIOCommand;
+import org.apache.giraph.ooc.OutOfCoreEngine;
+import org.apache.giraph.ooc.command.IOCommand;
+import org.apache.giraph.ooc.command.LoadPartitionIOCommand;
+import org.apache.giraph.ooc.command.StorePartitionIOCommand;
 import org.apache.log4j.Logger;
 
 import java.util.concurrent.atomic.AtomicInteger;
diff --git a/giraph-core/src/main/java/org/apache/giraph/ooc/OutOfCoreOracle.java b/giraph-core/src/main/java/org/apache/giraph/ooc/policy/OutOfCoreOracle.java
similarity index 93%
rename from giraph-core/src/main/java/org/apache/giraph/ooc/OutOfCoreOracle.java
rename to giraph-core/src/main/java/org/apache/giraph/ooc/policy/OutOfCoreOracle.java
index fa8e6bd..45b9914 100644
--- a/giraph-core/src/main/java/org/apache/giraph/ooc/OutOfCoreOracle.java
+++ b/giraph-core/src/main/java/org/apache/giraph/ooc/policy/OutOfCoreOracle.java
@@ -16,15 +16,19 @@
  * limitations under the License.
  */
 
-package org.apache.giraph.ooc;
+package org.apache.giraph.ooc.policy;
 
 import com.sun.management.GarbageCollectionNotificationInfo;
-import org.apache.giraph.ooc.io.IOCommand;
+import org.apache.giraph.ooc.command.IOCommand;
 
 /**
  * Interface for any out-of-core oracle. An out-of-core oracle is the brain of
  * the out-of-core mechanism, determining/deciding on out-of-core actions (load
  * or store) that should happen.
+ * Note: any class implementing this interface should have one and only one
+ *       constructor taking only two arguments of types
+ *       <code>ImmutableClassesGiraphConfiguration</code> and
+ *       <code>OutOfCoreEngine</code>
  */
 public interface OutOfCoreOracle {
   /**
diff --git a/giraph-core/src/main/java/org/apache/giraph/ooc/SimpleGCMonitoringOracle.java b/giraph-core/src/main/java/org/apache/giraph/ooc/policy/SimpleGCMonitoringOracle.java
similarity index 97%
rename from giraph-core/src/main/java/org/apache/giraph/ooc/SimpleGCMonitoringOracle.java
rename to giraph-core/src/main/java/org/apache/giraph/ooc/policy/SimpleGCMonitoringOracle.java
index 0dfc9de..477b3ec 100644
--- a/giraph-core/src/main/java/org/apache/giraph/ooc/SimpleGCMonitoringOracle.java
+++ b/giraph-core/src/main/java/org/apache/giraph/ooc/policy/SimpleGCMonitoringOracle.java
@@ -16,15 +16,17 @@
  * limitations under the License.
  */
 
-package org.apache.giraph.ooc;
+package org.apache.giraph.ooc.policy;
 
 import com.google.common.collect.Maps;
 import com.sun.management.GarbageCollectionNotificationInfo;
 import org.apache.giraph.conf.FloatConfOption;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.ooc.io.IOCommand;
-import org.apache.giraph.ooc.io.LoadPartitionIOCommand;
-import org.apache.giraph.ooc.io.WaitIOCommand;
+import org.apache.giraph.ooc.OutOfCoreEngine;
+import org.apache.giraph.ooc.OutOfCoreIOStatistics;
+import org.apache.giraph.ooc.command.IOCommand;
+import org.apache.giraph.ooc.command.LoadPartitionIOCommand;
+import org.apache.giraph.ooc.command.WaitIOCommand;
 import org.apache.log4j.Logger;
 
 import java.lang.management.MemoryUsage;
diff --git a/giraph-core/src/main/java/org/apache/giraph/ooc/ThresholdBasedOracle.java b/giraph-core/src/main/java/org/apache/giraph/ooc/policy/ThresholdBasedOracle.java
similarity index 98%
rename from giraph-core/src/main/java/org/apache/giraph/ooc/ThresholdBasedOracle.java
rename to giraph-core/src/main/java/org/apache/giraph/ooc/policy/ThresholdBasedOracle.java
index 3e05dce..ff2b3f7 100644
--- a/giraph-core/src/main/java/org/apache/giraph/ooc/ThresholdBasedOracle.java
+++ b/giraph-core/src/main/java/org/apache/giraph/ooc/policy/ThresholdBasedOracle.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.giraph.ooc;
+package org.apache.giraph.ooc.policy;
 
 import com.sun.management.GarbageCollectionNotificationInfo;
 import org.apache.giraph.comm.flow_control.CreditBasedFlowControl;
@@ -25,7 +25,8 @@
 import org.apache.giraph.conf.FloatConfOption;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.conf.LongConfOption;
-import org.apache.giraph.ooc.io.IOCommand;
+import org.apache.giraph.ooc.OutOfCoreEngine;
+import org.apache.giraph.ooc.command.IOCommand;
 import org.apache.giraph.utils.CallableFactory;
 import org.apache.giraph.utils.LogStacktraceCallable;
 import org.apache.giraph.utils.MemoryUtils;
diff --git a/giraph-core/src/main/java/org/apache/giraph/ooc/io/package-info.java b/giraph-core/src/main/java/org/apache/giraph/ooc/policy/package-info.java
similarity index 88%
rename from giraph-core/src/main/java/org/apache/giraph/ooc/io/package-info.java
rename to giraph-core/src/main/java/org/apache/giraph/ooc/policy/package-info.java
index 2230ec4..c58289f 100644
--- a/giraph-core/src/main/java/org/apache/giraph/ooc/io/package-info.java
+++ b/giraph-core/src/main/java/org/apache/giraph/ooc/policy/package-info.java
@@ -16,6 +16,6 @@
  * limitations under the License.
  */
 /**
- * Package of classes related to IO operations in out-of-core mechanism
+ * Package of classes related to out-of-core policy
  */
-package org.apache.giraph.ooc.io;
+package org.apache.giraph.ooc.policy;
diff --git a/giraph-core/src/main/java/org/apache/giraph/zk/ZooKeeperManager.java b/giraph-core/src/main/java/org/apache/giraph/zk/ZooKeeperManager.java
index d3ace99..c54e7b2 100644
--- a/giraph-core/src/main/java/org/apache/giraph/zk/ZooKeeperManager.java
+++ b/giraph-core/src/main/java/org/apache/giraph/zk/ZooKeeperManager.java
@@ -123,7 +123,7 @@
     this.context = context;
     this.conf = configuration;
     taskPartition = conf.getTaskPartition();
-    jobId = conf.get("mapred.job.id", "Unknown Job");
+    jobId = conf.getJobId();
     baseDirectory =
         new Path(ZOOKEEPER_MANAGER_DIRECTORY.getWithDefault(conf,
             getFinalZooKeeperPath()));
diff --git a/giraph-core/src/test/java/org/apache/giraph/partition/TestPartitionStores.java b/giraph-core/src/test/java/org/apache/giraph/partition/TestPartitionStores.java
index 3bb35eb..a7451bc 100644
--- a/giraph-core/src/test/java/org/apache/giraph/partition/TestPartitionStores.java
+++ b/giraph-core/src/test/java/org/apache/giraph/partition/TestPartitionStores.java
@@ -30,6 +30,7 @@
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.edge.EdgeFactory;
 import org.apache.giraph.graph.BasicComputation;
+import org.apache.giraph.graph.GraphTaskManager;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.io.formats.IdWithValueTextOutputFormat;
 import org.apache.giraph.io.formats.IntIntNullTextVertexInputFormat;
@@ -166,6 +167,9 @@
     ServerData<IntWritable, IntWritable, NullWritable>
         serverData = new ServerData<>(serviceWorker, conf, context);
     Mockito.when(serviceWorker.getServerData()).thenReturn(serverData);
+    GraphTaskManager<IntWritable, IntWritable, NullWritable>
+        graphTaskManager = new GraphTaskManager<>(context);
+    Mockito.when(serviceWorker.getGraphTaskManager()).thenReturn(graphTaskManager);
 
     DiskBackedPartitionStore<IntWritable, IntWritable, NullWritable>
         partitionStore =
@@ -192,6 +196,9 @@
     ServerData<IntWritable, IntWritable, NullWritable>
         serverData = new ServerData<>(serviceWorker, conf, context);
     Mockito.when(serviceWorker.getServerData()).thenReturn(serverData);
+    GraphTaskManager<IntWritable, IntWritable, NullWritable>
+        graphTaskManager = new GraphTaskManager<>(context);
+    Mockito.when(serviceWorker.getGraphTaskManager()).thenReturn(graphTaskManager);
 
     DiskBackedPartitionStore<IntWritable, IntWritable, NullWritable>
         partitionStore =
@@ -307,6 +314,9 @@
     ServerData<IntWritable, IntWritable, NullWritable>
         serverData = new ServerData<>(serviceWorker, conf, context);
     Mockito.when(serviceWorker.getServerData()).thenReturn(serverData);
+    GraphTaskManager<IntWritable, IntWritable, NullWritable>
+        graphTaskManager = new GraphTaskManager<>(context);
+    Mockito.when(serviceWorker.getGraphTaskManager()).thenReturn(graphTaskManager);
 
     DiskBackedPartitionStore<IntWritable, IntWritable, NullWritable>
         store =
diff --git a/giraph-rexster/giraph-rexster-io/src/main/java/org/apache/giraph/rexster/io/RexsterVertexOutputFormat.java b/giraph-rexster/giraph-rexster-io/src/main/java/org/apache/giraph/rexster/io/RexsterVertexOutputFormat.java
index 9af00e0..0e8d83e 100644
--- a/giraph-rexster/giraph-rexster-io/src/main/java/org/apache/giraph/rexster/io/RexsterVertexOutputFormat.java
+++ b/giraph-rexster/giraph-rexster-io/src/main/java/org/apache/giraph/rexster/io/RexsterVertexOutputFormat.java
@@ -186,7 +186,7 @@
       String id = context.getTaskAttemptID().toString();
       String zkBasePath = ZooKeeperManager.getBasePath(getConf()) +
         BspService.BASE_DIR + "/" +
-        getConf().get("mapred.job.id", "Unknown Job");
+        getConf().getJobId();
       prepareBarrier(zkBasePath);
       enterBarrier(zkBasePath, id);
       checkBarrier(zkBasePath, context);