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);