You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@giraph.apache.org by ni...@apache.org on 2013/01/04 21:52:39 UTC

[21/23] GIRAPH-409: Refactor / cleanups (nitay)

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyMasterClient.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyMasterClient.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyMasterClient.java
index c575172..86ea8dc 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyMasterClient.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyMasterClient.java
@@ -24,8 +24,8 @@ import org.apache.giraph.comm.MasterClient;
 import org.apache.giraph.comm.aggregators.AggregatorUtils;
 import org.apache.giraph.comm.aggregators.SendAggregatorCache;
 import org.apache.giraph.comm.requests.SendAggregatorsToOwnerRequest;
-import org.apache.giraph.graph.Aggregator;
-import org.apache.giraph.graph.WorkerInfo;
+import org.apache.giraph.aggregators.Aggregator;
+import org.apache.giraph.worker.WorkerInfo;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.util.Progressable;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerAggregatorRequestProcessor.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerAggregatorRequestProcessor.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerAggregatorRequestProcessor.java
index 6838321..cd24219 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerAggregatorRequestProcessor.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerAggregatorRequestProcessor.java
@@ -27,7 +27,7 @@ import org.apache.giraph.comm.aggregators.SendAggregatedValueCache;
 import org.apache.giraph.comm.requests.SendAggregatorsToMasterRequest;
 import org.apache.giraph.comm.requests.SendAggregatorsToWorkerRequest;
 import org.apache.giraph.comm.requests.SendWorkerAggregatorsRequest;
-import org.apache.giraph.graph.WorkerInfo;
+import org.apache.giraph.worker.WorkerInfo;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.util.Progressable;
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClient.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClient.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClient.java
index 4aff7b9..9e6ed66 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClient.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClient.java
@@ -23,8 +23,8 @@ import org.apache.giraph.bsp.CentralizedServiceWorker;
 import org.apache.giraph.comm.WorkerClient;
 import org.apache.giraph.comm.requests.WritableRequest;
 import org.apache.giraph.graph.TaskInfo;
-import org.apache.giraph.graph.WorkerInfo;
-import org.apache.giraph.graph.partition.PartitionOwner;
+import org.apache.giraph.worker.WorkerInfo;
+import org.apache.giraph.partition.PartitionOwner;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.Mapper;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClientRequestProcessor.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClientRequestProcessor.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClientRequestProcessor.java
index 0bc57d2..3db41bb 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClientRequestProcessor.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClientRequestProcessor.java
@@ -33,13 +33,13 @@ import org.apache.giraph.comm.requests.WorkerRequest;
 import org.apache.giraph.comm.requests.WritableRequest;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.graph.BspService;
+import org.apache.giraph.bsp.BspService;
 import org.apache.giraph.graph.Edge;
-import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.vertex.Vertex;
 import org.apache.giraph.graph.VertexMutations;
-import org.apache.giraph.graph.WorkerInfo;
-import org.apache.giraph.graph.partition.Partition;
-import org.apache.giraph.graph.partition.PartitionOwner;
+import org.apache.giraph.worker.WorkerInfo;
+import org.apache.giraph.partition.Partition;
+import org.apache.giraph.partition.PartitionOwner;
 import org.apache.giraph.metrics.GiraphMetrics;
 import org.apache.giraph.metrics.ValueGauge;
 import org.apache.giraph.utils.ByteArrayVertexIdMessages;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerServer.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerServer.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerServer.java
index 2f95a89..82ef831 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerServer.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerServer.java
@@ -34,10 +34,10 @@ import org.apache.giraph.comm.netty.handler.WorkerRequestServerHandler;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.graph.GraphState;
-import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.vertex.Vertex;
 import org.apache.giraph.graph.VertexMutations;
 import org.apache.giraph.graph.VertexResolver;
-import org.apache.giraph.graph.partition.Partition;
+import org.apache.giraph.partition.Partition;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.Mapper;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/MasterRequestServerHandler.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/MasterRequestServerHandler.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/MasterRequestServerHandler.java
index b3f0121..3e06026 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/MasterRequestServerHandler.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/MasterRequestServerHandler.java
@@ -20,7 +20,7 @@ package org.apache.giraph.comm.netty.handler;
 
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.comm.requests.MasterRequest;
-import org.apache.giraph.graph.MasterAggregatorHandler;
+import org.apache.giraph.master.MasterAggregatorHandler;
 import org.apache.giraph.graph.TaskInfo;
 
 /** Handler for requests on master */

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/RequestDecoder.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/RequestDecoder.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/RequestDecoder.java
index 0bf21e5..6eb6549 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/RequestDecoder.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/RequestDecoder.java
@@ -23,9 +23,9 @@ import org.apache.giraph.comm.netty.ByteCounter;
 import org.apache.giraph.comm.requests.RequestType;
 import org.apache.giraph.comm.requests.WritableRequest;
 import org.apache.giraph.utils.ReflectionUtils;
-import org.apache.giraph.utils.SystemTime;
-import org.apache.giraph.utils.Time;
-import org.apache.giraph.utils.Times;
+import org.apache.giraph.time.SystemTime;
+import org.apache.giraph.time.Time;
+import org.apache.giraph.time.Times;
 import org.apache.log4j.Logger;
 import org.jboss.netty.buffer.ChannelBuffer;
 import org.jboss.netty.buffer.ChannelBufferInputStream;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/RequestEncoder.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/RequestEncoder.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/RequestEncoder.java
index eade731..7fa0a4e 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/RequestEncoder.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/RequestEncoder.java
@@ -19,9 +19,9 @@
 package org.apache.giraph.comm.netty.handler;
 
 import org.apache.giraph.comm.requests.WritableRequest;
-import org.apache.giraph.utils.SystemTime;
-import org.apache.giraph.utils.Time;
-import org.apache.giraph.utils.Times;
+import org.apache.giraph.time.SystemTime;
+import org.apache.giraph.time.Time;
+import org.apache.giraph.time.Times;
 import org.apache.log4j.Logger;
 import org.jboss.netty.buffer.ChannelBuffer;
 import org.jboss.netty.buffer.ChannelBufferOutputStream;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/RequestInfo.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/RequestInfo.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/RequestInfo.java
index 97f5beb..1fa7c00 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/RequestInfo.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/RequestInfo.java
@@ -19,8 +19,8 @@
 package org.apache.giraph.comm.netty.handler;
 
 import org.apache.giraph.comm.requests.WritableRequest;
-import org.apache.giraph.utils.SystemTime;
-import org.apache.giraph.utils.Time;
+import org.apache.giraph.time.SystemTime;
+import org.apache.giraph.time.Time;
 import org.jboss.netty.channel.ChannelFuture;
 
 import java.net.InetSocketAddress;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/RequestServerHandler.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/RequestServerHandler.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/RequestServerHandler.java
index cd26ea2..bbf31c7 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/RequestServerHandler.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/RequestServerHandler.java
@@ -22,9 +22,9 @@ import org.apache.giraph.comm.requests.WritableRequest;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.graph.TaskInfo;
-import org.apache.giraph.utils.SystemTime;
-import org.apache.giraph.utils.Time;
-import org.apache.giraph.utils.Times;
+import org.apache.giraph.time.SystemTime;
+import org.apache.giraph.time.Time;
+import org.apache.giraph.time.Times;
 import org.apache.log4j.Logger;
 import org.jboss.netty.buffer.ChannelBuffer;
 import org.jboss.netty.buffer.ChannelBuffers;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/comm/requests/MasterRequest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/requests/MasterRequest.java b/giraph-core/src/main/java/org/apache/giraph/comm/requests/MasterRequest.java
index 18a23ea..7fedcc5 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/requests/MasterRequest.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/requests/MasterRequest.java
@@ -18,7 +18,7 @@
 
 package org.apache.giraph.comm.requests;
 
-import org.apache.giraph.graph.MasterAggregatorHandler;
+import org.apache.giraph.master.MasterAggregatorHandler;
 
 /**
  * Interface for requests sent to master to extend

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendAggregatorsToMasterRequest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendAggregatorsToMasterRequest.java b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendAggregatorsToMasterRequest.java
index 104e507..2a05192 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendAggregatorsToMasterRequest.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendAggregatorsToMasterRequest.java
@@ -18,7 +18,7 @@
 
 package org.apache.giraph.comm.requests;
 
-import org.apache.giraph.graph.MasterAggregatorHandler;
+import org.apache.giraph.master.MasterAggregatorHandler;
 
 import java.io.IOException;
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendAggregatorsToOwnerRequest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendAggregatorsToOwnerRequest.java b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendAggregatorsToOwnerRequest.java
index b102cfe..21b1b2d 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendAggregatorsToOwnerRequest.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendAggregatorsToOwnerRequest.java
@@ -21,7 +21,7 @@ package org.apache.giraph.comm.requests;
 import org.apache.giraph.comm.ServerData;
 import org.apache.giraph.comm.aggregators.AggregatorUtils;
 import org.apache.giraph.comm.aggregators.AllAggregatorServerData;
-import org.apache.giraph.graph.Aggregator;
+import org.apache.giraph.aggregators.Aggregator;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Writable;
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendAggregatorsToWorkerRequest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendAggregatorsToWorkerRequest.java b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendAggregatorsToWorkerRequest.java
index 0efd18d..7e84e17 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendAggregatorsToWorkerRequest.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendAggregatorsToWorkerRequest.java
@@ -21,7 +21,7 @@ package org.apache.giraph.comm.requests;
 import org.apache.giraph.comm.ServerData;
 import org.apache.giraph.comm.aggregators.AggregatorUtils;
 import org.apache.giraph.comm.aggregators.AllAggregatorServerData;
-import org.apache.giraph.graph.Aggregator;
+import org.apache.giraph.aggregators.Aggregator;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Writable;
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendVertexRequest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendVertexRequest.java b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendVertexRequest.java
index 4bc9daa..1de3cbb 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendVertexRequest.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendVertexRequest.java
@@ -19,7 +19,7 @@
 package org.apache.giraph.comm.requests;
 
 import org.apache.giraph.comm.ServerData;
-import org.apache.giraph.graph.partition.Partition;
+import org.apache.giraph.partition.Partition;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.log4j.Logger;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/conf/GiraphClasses.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphClasses.java b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphClasses.java
index d791e58..d2641f1 100644
--- a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphClasses.java
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphClasses.java
@@ -17,23 +17,23 @@
  */
 package org.apache.giraph.conf;
 
-import org.apache.giraph.graph.AggregatorWriter;
-import org.apache.giraph.graph.Combiner;
-import org.apache.giraph.graph.DefaultMasterCompute;
+import org.apache.giraph.aggregators.AggregatorWriter;
+import org.apache.giraph.combiner.Combiner;
+import org.apache.giraph.master.DefaultMasterCompute;
 import org.apache.giraph.graph.DefaultVertexResolver;
-import org.apache.giraph.graph.DefaultWorkerContext;
-import org.apache.giraph.graph.EdgeInputFormat;
-import org.apache.giraph.graph.MasterCompute;
-import org.apache.giraph.graph.TextAggregatorWriter;
-import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.graph.VertexInputFormat;
-import org.apache.giraph.graph.VertexOutputFormat;
+import org.apache.giraph.worker.DefaultWorkerContext;
+import org.apache.giraph.io.EdgeInputFormat;
+import org.apache.giraph.master.MasterCompute;
+import org.apache.giraph.aggregators.TextAggregatorWriter;
+import org.apache.giraph.vertex.Vertex;
+import org.apache.giraph.io.VertexInputFormat;
+import org.apache.giraph.io.VertexOutputFormat;
 import org.apache.giraph.graph.VertexResolver;
-import org.apache.giraph.graph.WorkerContext;
-import org.apache.giraph.graph.partition.GraphPartitionerFactory;
-import org.apache.giraph.graph.partition.HashPartitionerFactory;
-import org.apache.giraph.graph.partition.Partition;
-import org.apache.giraph.graph.partition.SimplePartition;
+import org.apache.giraph.worker.WorkerContext;
+import org.apache.giraph.partition.GraphPartitionerFactory;
+import org.apache.giraph.partition.HashPartitionerFactory;
+import org.apache.giraph.partition.Partition;
+import org.apache.giraph.partition.SimplePartition;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfiguration.java
----------------------------------------------------------------------
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 d5b9efe..df7b80e 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
@@ -18,17 +18,17 @@
 
 package org.apache.giraph.conf;
 
-import org.apache.giraph.graph.AggregatorWriter;
-import org.apache.giraph.graph.Combiner;
-import org.apache.giraph.graph.EdgeInputFormat;
-import org.apache.giraph.graph.MasterCompute;
-import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.graph.VertexInputFormat;
-import org.apache.giraph.graph.VertexOutputFormat;
+import org.apache.giraph.aggregators.AggregatorWriter;
+import org.apache.giraph.combiner.Combiner;
+import org.apache.giraph.io.EdgeInputFormat;
+import org.apache.giraph.master.MasterCompute;
+import org.apache.giraph.vertex.Vertex;
+import org.apache.giraph.io.VertexInputFormat;
+import org.apache.giraph.io.VertexOutputFormat;
 import org.apache.giraph.graph.VertexResolver;
-import org.apache.giraph.graph.WorkerContext;
-import org.apache.giraph.graph.partition.GraphPartitionerFactory;
-import org.apache.giraph.graph.partition.Partition;
+import org.apache.giraph.worker.WorkerContext;
+import org.apache.giraph.partition.GraphPartitionerFactory;
+import org.apache.giraph.partition.Partition;
 import org.apache.giraph.master.MasterObserver;
 import org.apache.hadoop.conf.Configuration;
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfiguration.java
----------------------------------------------------------------------
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 00e4135..f2c8701 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
@@ -18,20 +18,20 @@
 
 package org.apache.giraph.conf;
 
-import org.apache.giraph.graph.AggregatorWriter;
-import org.apache.giraph.graph.Combiner;
-import org.apache.giraph.graph.EdgeInputFormat;
+import org.apache.giraph.aggregators.AggregatorWriter;
+import org.apache.giraph.combiner.Combiner;
+import org.apache.giraph.io.EdgeInputFormat;
 import org.apache.giraph.graph.GraphState;
-import org.apache.giraph.graph.MasterCompute;
-import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.graph.VertexInputFormat;
-import org.apache.giraph.graph.VertexOutputFormat;
+import org.apache.giraph.master.MasterCompute;
+import org.apache.giraph.vertex.Vertex;
+import org.apache.giraph.io.VertexInputFormat;
+import org.apache.giraph.io.VertexOutputFormat;
 import org.apache.giraph.graph.VertexResolver;
-import org.apache.giraph.graph.WorkerContext;
-import org.apache.giraph.graph.partition.GraphPartitionerFactory;
-import org.apache.giraph.graph.partition.MasterGraphPartitioner;
-import org.apache.giraph.graph.partition.Partition;
-import org.apache.giraph.graph.partition.PartitionStats;
+import org.apache.giraph.worker.WorkerContext;
+import org.apache.giraph.partition.GraphPartitionerFactory;
+import org.apache.giraph.partition.MasterGraphPartitioner;
+import org.apache.giraph.partition.Partition;
+import org.apache.giraph.partition.PartitionStats;
 import org.apache.giraph.master.MasterObserver;
 import org.apache.giraph.utils.ExtendedByteArrayDataInput;
 import org.apache.giraph.utils.ExtendedByteArrayDataOutput;
@@ -90,7 +90,7 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
 
   /**
    * Get the user's subclassed
-   * {@link org.apache.giraph.graph.partition.GraphPartitionerFactory}.
+   * {@link org.apache.giraph.partition.GraphPartitionerFactory}.
    *
    * @return User's graph partitioner
    */
@@ -139,7 +139,7 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
 
   /**
    * Get the user's subclassed
-   * {@link org.apache.giraph.graph.VertexInputFormat}.
+   * {@link org.apache.giraph.io.VertexInputFormat}.
    *
    * @return User's vertex input format class
    */
@@ -162,7 +162,7 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
 
   /**
    * Get the user's subclassed
-   * {@link org.apache.giraph.graph.VertexOutputFormat}.
+   * {@link org.apache.giraph.io.VertexOutputFormat}.
    *
    * @return User's vertex output format class
    */
@@ -194,7 +194,7 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
 
   /**
    * Get the user's subclassed
-   * {@link org.apache.giraph.graph.EdgeInputFormat}.
+   * {@link org.apache.giraph.io.EdgeInputFormat}.
    *
    * @return User's edge input format class
    */
@@ -213,7 +213,7 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
   }
 
   /**
-   * Get the user's subclassed {@link org.apache.giraph.graph.AggregatorWriter}.
+   * Get the user's subclassed {@link AggregatorWriter}.
    *
    * @return User's aggregator writer class
    */
@@ -298,7 +298,7 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
   }
 
   /**
-   * Get the user's subclassed {@link org.apache.giraph.graph.MasterCompute}
+   * Get the user's subclassed {@link org.apache.giraph.master.MasterCompute}
    *
    * @return User's master class
    */
@@ -316,7 +316,7 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
   }
 
   /**
-   * Get the user's subclassed {@link org.apache.giraph.graph.Vertex}
+   * Get the user's subclassed {@link org.apache.giraph.vertex.Vertex}
    *
    * @return User's vertex class
    */

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/AggregatorsTestVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/AggregatorsTestVertex.java b/giraph-core/src/main/java/org/apache/giraph/examples/AggregatorsTestVertex.java
index 2233dcf..c00fda0 100644
--- a/giraph-core/src/main/java/org/apache/giraph/examples/AggregatorsTestVertex.java
+++ b/giraph-core/src/main/java/org/apache/giraph/examples/AggregatorsTestVertex.java
@@ -19,8 +19,8 @@
 package org.apache.giraph.examples;
 
 import org.apache.giraph.aggregators.LongSumAggregator;
-import org.apache.giraph.graph.DefaultMasterCompute;
-import org.apache.giraph.graph.EdgeListVertex;
+import org.apache.giraph.master.DefaultMasterCompute;
+import org.apache.giraph.vertex.EdgeListVertex;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.LongWritable;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/ConnectedComponentsVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/ConnectedComponentsVertex.java b/giraph-core/src/main/java/org/apache/giraph/examples/ConnectedComponentsVertex.java
index 0b2e547..b5c90e5 100644
--- a/giraph-core/src/main/java/org/apache/giraph/examples/ConnectedComponentsVertex.java
+++ b/giraph-core/src/main/java/org/apache/giraph/examples/ConnectedComponentsVertex.java
@@ -18,7 +18,7 @@
 
 package org.apache.giraph.examples;
 
-import org.apache.giraph.graph.IntIntNullIntVertex;
+import org.apache.giraph.vertex.IntIntNullIntVertex;
 import org.apache.hadoop.io.IntWritable;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/DoubleSumCombiner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/DoubleSumCombiner.java b/giraph-core/src/main/java/org/apache/giraph/examples/DoubleSumCombiner.java
deleted file mode 100644
index d0574d8..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/examples/DoubleSumCombiner.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.examples;
-
-import org.apache.giraph.graph.Combiner;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.LongWritable;
-
-/**
- * A combiner that sums double-valued messages
- */
-public class DoubleSumCombiner extends
-    Combiner<LongWritable, DoubleWritable> {
-  @Override
-  public void combine(LongWritable vertexIndex, DoubleWritable originalMessage,
-      DoubleWritable messageToCombine) {
-    originalMessage.set(originalMessage.get() + messageToCombine.get());
-  }
-
-  @Override
-  public DoubleWritable createInitialMessage() {
-    return new DoubleWritable(0);
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/GeneratedVertexReader.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/GeneratedVertexReader.java b/giraph-core/src/main/java/org/apache/giraph/examples/GeneratedVertexReader.java
index 662e759..26f970d 100644
--- a/giraph-core/src/main/java/org/apache/giraph/examples/GeneratedVertexReader.java
+++ b/giraph-core/src/main/java/org/apache/giraph/examples/GeneratedVertexReader.java
@@ -20,7 +20,7 @@ package org.apache.giraph.examples;
 
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.bsp.BspInputSplit;
-import org.apache.giraph.graph.VertexReader;
+import org.apache.giraph.io.VertexReader;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.InputSplit;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/IdentityVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/IdentityVertex.java b/giraph-core/src/main/java/org/apache/giraph/examples/IdentityVertex.java
index efa0d5a..7000035 100644
--- a/giraph-core/src/main/java/org/apache/giraph/examples/IdentityVertex.java
+++ b/giraph-core/src/main/java/org/apache/giraph/examples/IdentityVertex.java
@@ -18,7 +18,7 @@
 
 package org.apache.giraph.examples;
 
-import org.apache.giraph.graph.EdgeListVertex;
+import org.apache.giraph.vertex.EdgeListVertex;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/LongDoubleFloatDoubleTextInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/LongDoubleFloatDoubleTextInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/examples/LongDoubleFloatDoubleTextInputFormat.java
index b06f07f..53e83a4 100644
--- a/giraph-core/src/main/java/org/apache/giraph/examples/LongDoubleFloatDoubleTextInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/examples/LongDoubleFloatDoubleTextInputFormat.java
@@ -18,10 +18,10 @@
 
 package org.apache.giraph.examples;
 
-import org.apache.giraph.graph.BspUtils;
+import org.apache.giraph.bsp.BspUtils;
 import org.apache.giraph.graph.Edge;
-import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.io.TextVertexInputFormat;
+import org.apache.giraph.vertex.Vertex;
+import org.apache.giraph.io.formats.TextVertexInputFormat;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.LongWritable;
@@ -35,7 +35,7 @@ import java.util.List;
 import java.util.regex.Pattern;
 
 /**
- * Simple text-based {@link org.apache.giraph.graph.VertexInputFormat} for
+ * Simple text-based {@link org.apache.giraph.io.VertexInputFormat} for
  * unweighted graphs with long ids. Each line consists of: vertex neighbor1
  * neighbor2 ...
  */

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/MinimumDoubleCombiner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/MinimumDoubleCombiner.java b/giraph-core/src/main/java/org/apache/giraph/examples/MinimumDoubleCombiner.java
deleted file mode 100644
index 9da10b4..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/examples/MinimumDoubleCombiner.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.examples;
-
-import org.apache.giraph.graph.Combiner;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.LongWritable;
-
-/**
- * Combiner which finds the minimum of {@link DoubleWritable}.
- */
-public class MinimumDoubleCombiner extends
-    Combiner<LongWritable, DoubleWritable> {
-  @Override
-  public void combine(LongWritable vertexIndex, DoubleWritable originalMessage,
-      DoubleWritable messageToCombine) {
-    if (originalMessage.get() > messageToCombine.get()) {
-      originalMessage.set(messageToCombine.get());
-    }
-  }
-
-  @Override
-  public DoubleWritable createInitialMessage() {
-    return new DoubleWritable(Double.MAX_VALUE);
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/MinimumIntCombiner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/MinimumIntCombiner.java b/giraph-core/src/main/java/org/apache/giraph/examples/MinimumIntCombiner.java
deleted file mode 100644
index 82e45a6..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/examples/MinimumIntCombiner.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.examples;
-
-import org.apache.giraph.graph.Combiner;
-import org.apache.hadoop.io.IntWritable;
-
-/**
- * {@link Combiner} that finds the minimum {@link IntWritable}
- */
-public class MinimumIntCombiner
-    extends Combiner<IntWritable, IntWritable> {
-  @Override
-  public void combine(IntWritable vertexIndex, IntWritable originalMessage,
-      IntWritable messageToCombine) {
-    if (originalMessage.get() > messageToCombine.get()) {
-      originalMessage.set(messageToCombine.get());
-    }
-  }
-
-  @Override
-  public IntWritable createInitialMessage() {
-    return new IntWritable(Integer.MAX_VALUE);
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/NormalizingLongDoubleFloatDoubleTextInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/NormalizingLongDoubleFloatDoubleTextInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/examples/NormalizingLongDoubleFloatDoubleTextInputFormat.java
index 300fbb6..65bc636 100644
--- a/giraph-core/src/main/java/org/apache/giraph/examples/NormalizingLongDoubleFloatDoubleTextInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/examples/NormalizingLongDoubleFloatDoubleTextInputFormat.java
@@ -18,10 +18,10 @@
 
 package org.apache.giraph.examples;
 
-import org.apache.giraph.graph.BspUtils;
+import org.apache.giraph.bsp.BspUtils;
 import org.apache.giraph.graph.Edge;
-import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.io.TextVertexInputFormat;
+import org.apache.giraph.vertex.Vertex;
+import org.apache.giraph.io.formats.TextVertexInputFormat;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.LongWritable;
@@ -36,7 +36,7 @@ import java.util.List;
 import java.util.regex.Pattern;
 
 /**
- * Simple text-based {@link org.apache.giraph.graph.VertexInputFormat} for
+ * Simple text-based {@link org.apache.giraph.io.VertexInputFormat} for
  * unweighted graphs with long ids. Each line consists of: vertex
  * neighbor1:weight1 neighbor2:weight2 ...
  */

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/RandomWalkVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/RandomWalkVertex.java b/giraph-core/src/main/java/org/apache/giraph/examples/RandomWalkVertex.java
index c785870..3c0abe7 100644
--- a/giraph-core/src/main/java/org/apache/giraph/examples/RandomWalkVertex.java
+++ b/giraph-core/src/main/java/org/apache/giraph/examples/RandomWalkVertex.java
@@ -21,9 +21,9 @@ package org.apache.giraph.examples;
 import java.io.IOException;
 
 import org.apache.giraph.aggregators.DoubleSumAggregator;
-import org.apache.giraph.graph.DefaultMasterCompute;
+import org.apache.giraph.master.DefaultMasterCompute;
 import org.apache.giraph.graph.Edge;
-import org.apache.giraph.graph.LongDoubleFloatDoubleEdgeListVertex;
+import org.apache.giraph.vertex.LongDoubleFloatDoubleEdgeListVertex;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.LongWritable;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/RandomWalkWorkerContext.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/RandomWalkWorkerContext.java b/giraph-core/src/main/java/org/apache/giraph/examples/RandomWalkWorkerContext.java
index 9e4e743..5cff23f 100644
--- a/giraph-core/src/main/java/org/apache/giraph/examples/RandomWalkWorkerContext.java
+++ b/giraph-core/src/main/java/org/apache/giraph/examples/RandomWalkWorkerContext.java
@@ -23,7 +23,7 @@ import java.io.IOException;
 import java.io.InputStreamReader;
 import java.util.Set;
 
-import org.apache.giraph.graph.WorkerContext;
+import org.apache.giraph.worker.WorkerContext;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.filecache.DistributedCache;
 import org.apache.hadoop.fs.FileSystem;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/SimpleAggregatorWriter.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleAggregatorWriter.java b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleAggregatorWriter.java
index e22ecba..1887621 100644
--- a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleAggregatorWriter.java
+++ b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleAggregatorWriter.java
@@ -21,7 +21,7 @@ package org.apache.giraph.examples;
 import java.io.IOException;
 import java.util.Map.Entry;
 
-import org.apache.giraph.graph.AggregatorWriter;
+import org.apache.giraph.aggregators.AggregatorWriter;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/SimpleCheckpointVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleCheckpointVertex.java b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleCheckpointVertex.java
index 5da66aa..b23a163 100644
--- a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleCheckpointVertex.java
+++ b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleCheckpointVertex.java
@@ -24,13 +24,13 @@ import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.PosixParser;
 import org.apache.giraph.aggregators.LongSumAggregator;
-import org.apache.giraph.graph.DefaultMasterCompute;
+import org.apache.giraph.master.DefaultMasterCompute;
 import org.apache.giraph.graph.Edge;
-import org.apache.giraph.graph.EdgeListVertex;
+import org.apache.giraph.vertex.EdgeListVertex;
 import org.apache.giraph.graph.GiraphJob;
-import org.apache.giraph.graph.WorkerContext;
-import org.apache.giraph.io.GeneratedVertexInputFormat;
-import org.apache.giraph.io.IdWithValueTextOutputFormat;
+import org.apache.giraph.worker.WorkerContext;
+import org.apache.giraph.io.formats.GeneratedVertexInputFormat;
+import org.apache.giraph.io.formats.IdWithValueTextOutputFormat;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.FloatWritable;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/SimpleCombinerVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleCombinerVertex.java b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleCombinerVertex.java
index 37d50b2..be0f16c 100644
--- a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleCombinerVertex.java
+++ b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleCombinerVertex.java
@@ -18,7 +18,7 @@
 
 package org.apache.giraph.examples;
 
-import org.apache.giraph.graph.EdgeListVertex;
+import org.apache.giraph.vertex.EdgeListVertex;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/SimpleFailVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleFailVertex.java b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleFailVertex.java
index 04e5109..521ed97 100644
--- a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleFailVertex.java
+++ b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleFailVertex.java
@@ -18,7 +18,7 @@
 
 package org.apache.giraph.examples;
 
-import org.apache.giraph.graph.EdgeListVertex;
+import org.apache.giraph.vertex.EdgeListVertex;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.LongWritable;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/SimpleInDegreeCountVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleInDegreeCountVertex.java b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleInDegreeCountVertex.java
index 00baf9a..605d9d5 100644
--- a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleInDegreeCountVertex.java
+++ b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleInDegreeCountVertex.java
@@ -21,7 +21,7 @@ package org.apache.giraph.examples;
 import org.apache.giraph.graph.Edge;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.DoubleWritable;
-import org.apache.giraph.graph.EdgeListVertex;
+import org.apache.giraph.vertex.EdgeListVertex;
 
 /**
  * Simple function to return the out degree for each vertex.

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/SimpleMasterComputeVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleMasterComputeVertex.java b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleMasterComputeVertex.java
index a70e5cc..e481ff7 100644
--- a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleMasterComputeVertex.java
+++ b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleMasterComputeVertex.java
@@ -19,9 +19,9 @@
 package org.apache.giraph.examples;
 
 import org.apache.giraph.aggregators.DoubleOverwriteAggregator;
-import org.apache.giraph.graph.DefaultMasterCompute;
-import org.apache.giraph.graph.LongDoubleFloatDoubleVertex;
-import org.apache.giraph.graph.WorkerContext;
+import org.apache.giraph.master.DefaultMasterCompute;
+import org.apache.giraph.vertex.LongDoubleFloatDoubleVertex;
+import org.apache.giraph.worker.WorkerContext;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.log4j.Logger;
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/SimpleMsgVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleMsgVertex.java b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleMsgVertex.java
index a822749..8016490 100644
--- a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleMsgVertex.java
+++ b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleMsgVertex.java
@@ -18,7 +18,7 @@
 
 package org.apache.giraph.examples;
 
-import org.apache.giraph.graph.EdgeListVertex;
+import org.apache.giraph.vertex.EdgeListVertex;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/SimpleMutateGraphVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleMutateGraphVertex.java b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleMutateGraphVertex.java
index 0a7589d..5d7deb7 100644
--- a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleMutateGraphVertex.java
+++ b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleMutateGraphVertex.java
@@ -19,8 +19,8 @@
 package org.apache.giraph.examples;
 
 import org.apache.giraph.graph.Edge;
-import org.apache.giraph.graph.EdgeListVertex;
-import org.apache.giraph.graph.WorkerContext;
+import org.apache.giraph.vertex.EdgeListVertex;
+import org.apache.giraph.worker.WorkerContext;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.LongWritable;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/SimpleOutDegreeCountVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleOutDegreeCountVertex.java b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleOutDegreeCountVertex.java
index 4320564..1e010a1 100644
--- a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleOutDegreeCountVertex.java
+++ b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleOutDegreeCountVertex.java
@@ -20,7 +20,7 @@ package org.apache.giraph.examples;
 
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.DoubleWritable;
-import org.apache.giraph.graph.EdgeListVertex;
+import org.apache.giraph.vertex.EdgeListVertex;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/SimplePageRankVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/SimplePageRankVertex.java b/giraph-core/src/main/java/org/apache/giraph/examples/SimplePageRankVertex.java
index 5fd834b..c3b79b4 100644
--- a/giraph-core/src/main/java/org/apache/giraph/examples/SimplePageRankVertex.java
+++ b/giraph-core/src/main/java/org/apache/giraph/examples/SimplePageRankVertex.java
@@ -21,14 +21,14 @@ package org.apache.giraph.examples;
 import org.apache.giraph.aggregators.DoubleMaxAggregator;
 import org.apache.giraph.aggregators.DoubleMinAggregator;
 import org.apache.giraph.aggregators.LongSumAggregator;
-import org.apache.giraph.graph.DefaultMasterCompute;
+import org.apache.giraph.master.DefaultMasterCompute;
 import org.apache.giraph.graph.Edge;
-import org.apache.giraph.graph.LongDoubleFloatDoubleVertex;
-import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.graph.VertexReader;
-import org.apache.giraph.graph.WorkerContext;
-import org.apache.giraph.io.GeneratedVertexInputFormat;
-import org.apache.giraph.io.TextVertexOutputFormat;
+import org.apache.giraph.vertex.LongDoubleFloatDoubleVertex;
+import org.apache.giraph.vertex.Vertex;
+import org.apache.giraph.io.VertexReader;
+import org.apache.giraph.worker.WorkerContext;
+import org.apache.giraph.io.formats.GeneratedVertexInputFormat;
+import org.apache.giraph.io.formats.TextVertexOutputFormat;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.LongWritable;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/SimpleShortestPathsVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleShortestPathsVertex.java b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleShortestPathsVertex.java
index 26d7edb..1bec7eb 100644
--- a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleShortestPathsVertex.java
+++ b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleShortestPathsVertex.java
@@ -19,7 +19,7 @@
 package org.apache.giraph.examples;
 
 import org.apache.giraph.graph.Edge;
-import org.apache.giraph.graph.EdgeListVertex;
+import org.apache.giraph.vertex.EdgeListVertex;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.LongWritable;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/SimpleSumCombiner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleSumCombiner.java b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleSumCombiner.java
deleted file mode 100644
index bc97ae0..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleSumCombiner.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.examples;
-
-import org.apache.giraph.graph.Combiner;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
-
-/**
- * Combiner which sums up {@link IntWritable} message values.
- */
-public class SimpleSumCombiner
-    extends Combiner<LongWritable, IntWritable> {
-
-  @Override
-  public void combine(LongWritable vertexIndex, IntWritable originalMessage,
-      IntWritable messageToCombine) {
-    originalMessage.set(originalMessage.get() + messageToCombine.get());
-  }
-
-  @Override
-  public IntWritable createInitialMessage() {
-    return new IntWritable(0);
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/SimpleSuperstepVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleSuperstepVertex.java b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleSuperstepVertex.java
index b166ce0..2837a57 100644
--- a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleSuperstepVertex.java
+++ b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleSuperstepVertex.java
@@ -19,11 +19,11 @@
 package org.apache.giraph.examples;
 
 import org.apache.giraph.graph.Edge;
-import org.apache.giraph.graph.EdgeListVertex;
-import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.graph.VertexReader;
-import org.apache.giraph.io.GeneratedVertexInputFormat;
-import org.apache.giraph.io.TextVertexOutputFormat;
+import org.apache.giraph.vertex.EdgeListVertex;
+import org.apache.giraph.vertex.Vertex;
+import org.apache.giraph.io.VertexReader;
+import org.apache.giraph.io.formats.GeneratedVertexInputFormat;
+import org.apache.giraph.io.formats.TextVertexOutputFormat;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/SimpleTextVertexOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleTextVertexOutputFormat.java b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleTextVertexOutputFormat.java
index b610bcf..a57c6d2 100644
--- a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleTextVertexOutputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleTextVertexOutputFormat.java
@@ -18,8 +18,8 @@
 
 package org.apache.giraph.examples;
 
-import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.io.TextVertexOutputFormat;
+import org.apache.giraph.vertex.Vertex;
+import org.apache.giraph.io.formats.TextVertexOutputFormat;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/SimpleTriangleClosingVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleTriangleClosingVertex.java b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleTriangleClosingVertex.java
index 4cb7891..484dd8a 100644
--- a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleTriangleClosingVertex.java
+++ b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleTriangleClosingVertex.java
@@ -19,7 +19,7 @@
 package org.apache.giraph.examples;
 
 import org.apache.giraph.graph.Edge;
-import org.apache.giraph.graph.EdgeListVertex;
+import org.apache.giraph.vertex.EdgeListVertex;
 import org.apache.giraph.utils.ArrayListWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.NullWritable;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/SimpleVertexWithWorkerContext.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleVertexWithWorkerContext.java b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleVertexWithWorkerContext.java
index 4e8df31..b7605bb 100644
--- a/giraph-core/src/main/java/org/apache/giraph/examples/SimpleVertexWithWorkerContext.java
+++ b/giraph-core/src/main/java/org/apache/giraph/examples/SimpleVertexWithWorkerContext.java
@@ -20,9 +20,9 @@ package org.apache.giraph.examples;
 
 import org.apache.giraph.examples.SimpleSuperstepVertex.
     SimpleSuperstepVertexInputFormat;
-import org.apache.giraph.graph.EdgeListVertex;
+import org.apache.giraph.vertex.EdgeListVertex;
 import org.apache.giraph.graph.GiraphJob;
-import org.apache.giraph.graph.WorkerContext;
+import org.apache.giraph.worker.WorkerContext;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/VerifyMessage.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/VerifyMessage.java b/giraph-core/src/main/java/org/apache/giraph/examples/VerifyMessage.java
index 0d57062..2914360 100644
--- a/giraph-core/src/main/java/org/apache/giraph/examples/VerifyMessage.java
+++ b/giraph-core/src/main/java/org/apache/giraph/examples/VerifyMessage.java
@@ -19,10 +19,10 @@
 package org.apache.giraph.examples;
 
 import org.apache.giraph.aggregators.LongSumAggregator;
-import org.apache.giraph.graph.DefaultMasterCompute;
+import org.apache.giraph.master.DefaultMasterCompute;
 import org.apache.giraph.graph.Edge;
-import org.apache.giraph.graph.EdgeListVertex;
-import org.apache.giraph.graph.WorkerContext;
+import org.apache.giraph.vertex.EdgeListVertex;
+import org.apache.giraph.worker.WorkerContext;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/examples/VertexWithDoubleValueFloatEdgeTextOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/examples/VertexWithDoubleValueFloatEdgeTextOutputFormat.java b/giraph-core/src/main/java/org/apache/giraph/examples/VertexWithDoubleValueFloatEdgeTextOutputFormat.java
index 446eff1..ef58bb8 100644
--- a/giraph-core/src/main/java/org/apache/giraph/examples/VertexWithDoubleValueFloatEdgeTextOutputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/examples/VertexWithDoubleValueFloatEdgeTextOutputFormat.java
@@ -20,8 +20,8 @@ package org.apache.giraph.examples;
 
 import java.io.IOException;
 
-import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.io.TextVertexOutputFormat;
+import org.apache.giraph.vertex.Vertex;
+import org.apache.giraph.io.formats.TextVertexOutputFormat;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.LongWritable;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/graph/AddressesAndPartitionsWritable.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/AddressesAndPartitionsWritable.java b/giraph-core/src/main/java/org/apache/giraph/graph/AddressesAndPartitionsWritable.java
index d154034..990b04e 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/AddressesAndPartitionsWritable.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/AddressesAndPartitionsWritable.java
@@ -18,7 +18,9 @@
 
 package org.apache.giraph.graph;
 
-import org.apache.giraph.graph.partition.PartitionOwner;
+import org.apache.giraph.partition.PartitionOwner;
+import org.apache.giraph.master.MasterInfo;
+import org.apache.giraph.worker.WorkerInfo;
 import org.apache.hadoop.io.Writable;
 
 import com.google.common.collect.Lists;

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/graph/Aggregator.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/Aggregator.java b/giraph-core/src/main/java/org/apache/giraph/graph/Aggregator.java
deleted file mode 100644
index 3fe4c38..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/graph/Aggregator.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * 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.graph;
-
-import org.apache.hadoop.io.Writable;
-
-/**
- * Interface for Aggregator.  Allows aggregate operations for all vertices
- * in a given superstep.
- *
- * @param <A> Aggregated value
- */
-public interface Aggregator<A extends Writable> {
-  /**
-   * Add a new value.
-   * Needs to be commutative and associative
-   *
-   * @param value Value to be aggregated.
-   */
-  void aggregate(A value);
-
-  /**
-   * Return new aggregated value which is neutral to aggregate operation.
-   * Must be changeable without affecting internals of Aggregator
-   *
-   * @return Neutral value
-   */
-  A createInitialValue();
-
-  /**
-   * Return current aggregated value.
-   * Needs to be initialized if aggregate or setAggregatedValue
-   * have not been called before.
-   *
-   * @return Aggregated
-   */
-  A getAggregatedValue();
-
-  /**
-   * Set aggregated value.
-   * Can be used for initialization or reset.
-   *
-   * @param value Value to be set.
-   */
-  void setAggregatedValue(A value);
-
-  /**
-   * Reset the value of aggregator to neutral value
-   */
-  void reset();
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/graph/AggregatorWrapper.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/AggregatorWrapper.java b/giraph-core/src/main/java/org/apache/giraph/graph/AggregatorWrapper.java
deleted file mode 100644
index a59a5e0..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/graph/AggregatorWrapper.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/*
- * 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.graph;
-
-import org.apache.hadoop.io.Writable;
-
-/**
- * Wrapper for aggregators. Keeps two instances of an aggregator - one for
- * the value from previous super step, and one for the value which is being
- * generated in current super step.
- *
- * @param <A> Aggregated value
- */
-public class AggregatorWrapper<A extends Writable> {
-  /** False iff aggregator should be reset at the end of each super step */
-  private final boolean persistent;
-  /** Value aggregated in previous super step */
-  private A previousAggregatedValue;
-  /** Aggregator for next super step */
-  private final Aggregator<A> currentAggregator;
-  /** Whether anyone changed current value since the moment it was reset */
-  private boolean changed;
-
-  /**
-   * @param aggregatorClass Class type of the aggregator
-   * @param persistent      False iff aggregator should be reset at the end of
-   *                        each super step
-   * @throws IllegalAccessException
-   * @throws InstantiationException
-   */
-  public AggregatorWrapper(Class<? extends Aggregator<A>> aggregatorClass,
-      boolean persistent) throws IllegalAccessException,
-      InstantiationException {
-    this.persistent = persistent;
-    currentAggregator = aggregatorClass.newInstance();
-    changed = false;
-    previousAggregatedValue = currentAggregator.createInitialValue();
-  }
-
-  /**
-   * Get aggregated value from previous super step
-   *
-   * @return Aggregated value from previous super step
-   */
-  public A getPreviousAggregatedValue() {
-    return previousAggregatedValue;
-  }
-
-  /**
-   * Set aggregated value for previous super step
-   *
-   * @param value Aggregated value to set
-   */
-  public void setPreviousAggregatedValue(A value) {
-    previousAggregatedValue = value;
-  }
-
-  /**
-   * Check if aggregator is persistent
-   *
-   * @return False iff aggregator should be reset at the end of each super step
-   */
-  public boolean isPersistent() {
-    return persistent;
-  }
-
-  /**
-   * Check if current aggregator was changed
-   *
-   * @return Whether anyone changed current value since the moment it was reset
-   */
-  public boolean isChanged() {
-    return changed;
-  }
-
-  /**
-   * Add a new value to current aggregator
-   *
-   * @param value Value to be aggregated
-   */
-  public synchronized void aggregateCurrent(A value) {
-    changed = true;
-    currentAggregator.aggregate(value);
-  }
-
-  /**
-   * Get current aggregated value
-   *
-   * @return Current aggregated value
-   */
-  public A getCurrentAggregatedValue() {
-    return currentAggregator.getAggregatedValue();
-  }
-
-  /**
-   * Set aggregated value of current aggregator
-   *
-   * @param value Value to set it to
-   */
-  public void setCurrentAggregatedValue(A value) {
-    changed = true;
-    currentAggregator.setAggregatedValue(value);
-  }
-
-  /**
-   * Reset the value of current aggregator to neutral value
-   */
-  public void resetCurrentAggregator() {
-    changed = false;
-    currentAggregator.reset();
-  }
-
-  /**
-   * Return new aggregated value which is neutral to aggregate operation
-   *
-   * @return Neutral value
-   */
-  public A createInitialValue() {
-    return currentAggregator.createInitialValue();
-  }
-
-  /**
-   * Get class of wrapped aggregator
-   *
-   * @return Aggregator class
-   */
-  public Class<? extends Aggregator> getAggregatorClass() {
-    return currentAggregator.getClass();
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/graph/AggregatorWriter.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/AggregatorWriter.java b/giraph-core/src/main/java/org/apache/giraph/graph/AggregatorWriter.java
deleted file mode 100644
index fbb1855..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/graph/AggregatorWriter.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * 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.graph;
-
-import java.io.IOException;
-import java.util.Map.Entry;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapreduce.Mapper.Context;
-
-/**
- *  An AggregatorWriter is used to export Aggregators during or at the end of
- *  each computation. It runs on the master and it's called at the end of each
- *  superstep. The special signal {@link AggregatorWriter#LAST_SUPERSTEP} is
- *  passed to {@link AggregatorWriter#writeAggregator(Iterable, long)} as the
- *  superstep value to signal the end of computation.
- */
-public interface AggregatorWriter {
-  /** Signal for last superstep */
-  int LAST_SUPERSTEP = -1;
-
-  /**
-   * The method is called at the initialization of the AggregatorWriter.
-   * More precisely, the aggregatorWriter is initialized each time a new
-   * master is elected.
-   *
-   * @param context Mapper Context where the master is running on
-   * @param applicationAttempt ID of the applicationAttempt, used to
-   *        disambiguate aggregator writes for different attempts
-   * @throws IOException
-   */
-  @SuppressWarnings("rawtypes")
-  void initialize(Context context, long applicationAttempt) throws IOException;
-
-  /**
-   * The method is called at the end of each superstep. The user might decide
-   * whether to write the aggregators values for the current superstep. For
-   * the last superstep, {@link AggregatorWriter#LAST_SUPERSTEP} is passed.
-   *
-   * @param aggregatorMap Map from aggregator name to aggregator value
-   * @param superstep Current superstep
-   * @throws IOException
-   */
-  void writeAggregator(
-      Iterable<Entry<String, Writable>> aggregatorMap,
-      long superstep) throws IOException;
-
-  /**
-   * The method is called at the end of a successful computation. The method
-   * is not called when the job fails and a new master is elected. For this
-   * reason it's advised to flush data at the end of
-   * {@link AggregatorWriter#writeAggregator(Iterable, long)}.
-   *
-   * @throws IOException
-   */
-  void close() throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/1684891e/giraph-core/src/main/java/org/apache/giraph/graph/BasicVertexValueReader.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/BasicVertexValueReader.java b/giraph-core/src/main/java/org/apache/giraph/graph/BasicVertexValueReader.java
deleted file mode 100644
index 4d620fb..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/graph/BasicVertexValueReader.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.graph;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-
-import java.io.IOException;
-
-/**
- * Abstract base class for VertexValueReader.
- *
- * @param <I> Vertex id
- * @param <V> Vertex data
- * @param <E> Edge data
- * @param <M> Message data
- */
-public abstract class BasicVertexValueReader<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    implements VertexReader<I, V, E, M> {
-  /**
-   * User-defined method to extract the vertex id.
-   *
-   * @return The vertex id
-   * @throws java.io.IOException
-   * @throws InterruptedException
-   */
-  public abstract I getCurrentVertexId() throws IOException,
-      InterruptedException;
-
-  /**
-   * User-defined method to extract the vertex value.
-   *
-   * @return The vertex value
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public abstract V getCurrentVertexValue() throws IOException,
-      InterruptedException;
-}